Skip to content

View Spec implementation #331

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 30 commits into from
Jul 28, 2024
Merged

View Spec implementation #331

merged 30 commits into from
Jul 28, 2024

Conversation

c-thiel
Copy link
Collaborator

@c-thiel c-thiel commented Apr 11, 2024

Implementation of the Iceberg V1 ViewMetadata.

My plan of heading forward:

  1. Get this PR merged which just contains the ViewSpec and is not very usable on its own. (Similar to TableSpec which is missing the complete builder also)
  2. Add TableMetadataBuilder including partition binding in separate PR
  3. Add full ViewMetadataBuilder in separate PR

@c-thiel c-thiel mentioned this pull request Apr 11, 2024
@ZENOTME
Copy link
Contributor

ZENOTME commented Apr 11, 2024

Thanks! Nice work! @c-thiel

@c-thiel
Copy link
Collaborator Author

c-thiel commented Apr 12, 2024

@ZENOTME one feature I did not implement yet is respecting "version.history.num-entries" as mentioned in the View Spec. I noticed that the table implementation also doesn't implement it. I don't think it matters much unless one is implementing a catalog, but I wanted to mention it.

@Fokko
Copy link
Contributor

Fokko commented Apr 15, 2024

Thanks for working on this @c-thiel. Out of curiosity, and probably this is a broader discussion, but currently, the view spec is around SQL representation. This might not be the most natural fit with dataframe-based systems. Do you have any thoughts on how to represent views in iceberg-rust?

@c-thiel
Copy link
Collaborator Author

c-thiel commented Apr 18, 2024

@Fokko thats is a hard topic.

The idealist in me would like to eventually see something like substrait beeing used.
Adoption of the project is very slow across engines though.

Maybe the more practical short-term solution would be to stick to representations and even limit to one dialect - the one of the "writer", including its technology. Then each client can decide if there is a transpiler good enough for that dialect.
Funnily enough exactly this ambiguity - that there could be different dialects in the representation (say spark and trino) - gave me a lot of headaches. Which should I use if I am Datafusion? From which should I transpile? What to do if they are not identical?

Sticking to multiple representations is in my opinion only an option if we let the catalog handle this complexity.
Unless substrait gets a significant boost, I would assume that the following approach presents the most stable API (Assuming there is only REST Catalog left ;) ):

  • A client would create a view as it does today - however limited to a single representation
  • Upon read, a client would additionally present a set of "understood" dialects. The catalog can either serve them or it can't.

This way a Rest-Catalog could even switch to Substrait internally in the future and start now with storing the presented SQL "as-is" without beeing able to serve any other dialect.

With SQLMesh coming up and also Substrait continuing there is at least some development in the area.

@c-thiel c-thiel changed the title WIP: View Spec implementation View Spec implementation May 4, 2024
@c-thiel
Copy link
Collaborator Author

c-thiel commented May 4, 2024

@Fokko from my side this is good to merge - types are complete and tests are passing.

@c-thiel
Copy link
Collaborator Author

c-thiel commented Jun 20, 2024

@Fokko @ZENOTME
are there any points open from your side that prevent us to merge the View Spec?
If so, please let me know :)

@Fokko
Copy link
Contributor

Fokko commented Jun 20, 2024

The idealist in me would like to eventually see something like substrait beeing used.

There is an open discussion on evolving the spec into something like that. I think transpiling SQL will never work in the long run because there are so many variations and extensions to the SQL dialect (custom UDFs and such), maybe you can reach 90% of the SQL which is good enough.

@nastra would you have time to go over this PR?

@@ -0,0 +1,58 @@
{
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it would be good to have more JSON files for testing (not just the happy case)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also in general I think this needs more extensive testing. For example on the java side we have https://github.com/apache/iceberg/tree/5ea78e3fbe5d8c9c846a1b86bcd2b77d13a31acd/core/src/test/java/org/apache/iceberg/view.

Specifically in TestViewMetadata we have a bunch of test cases that would be good to cover here as well

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, thanks for the feedback @nastra. I'll add some more tests next week.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@nastra I added more tests that cover non-happy cases.
However, there isn't much in https://github.com/apache/iceberg/tree/5ea78e3fbe5d8c9c846a1b86bcd2b77d13a31acd/core/src/test/java/org/apache/iceberg/view we can use for now. The ViewMetadataBuilder I started with is on the same level as the TableMetadataBuilder - it can do almost nothing except changing the UUID. The functions tested in the java part are simply not implemented yet. We do have a more complete implementation over at our Iceberg catalog and want to PR them here (lakekeeper/lakekeeper#46), I just want to start with the foundation before heading into the more controversial builder bits.
Would this be alright for you?

Besides the Metadata-Builder, I would like to create 2 follow-up issues:

  • Currently in Snapshots we use a fallible method for the timestamp() method. For views I am using a non-fallible version, creating a small inconsistency. I would create a new issues to remove unwrap from timestamps where they are used.
  • Allow for empty namespaces, following our discussion below. I am not sure exactly how to handle this, but right now Rust does not allow the creation of the empty Namespace as used in spark (
    if names.is_empty() {
    ).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

However, there isn't much in https://github.com/apache/iceberg/tree/5ea78e3fbe5d8c9c846a1b86bcd2b77d13a31acd/core/src/test/java/org/apache/iceberg/view we can use for now.

In the java code we typically don't construct the full JSON just to test something, but there are plenty of tests in TestViewMetadata to make sure schema / view version re-assignment / de-duplication happens or SQL dialect deduplication behaves correctly.

I just want to start with the foundation before heading into the more controversial builder bits.

I was merely asked to look at this view implementation since I did the Java implementation, but I'm not involved in iceberg-rust, so I'll let others decide how to proceed with adding views to iceberg-rust and what to include in this PR vs in follow-ups.
Regarding follow-ups, I think a few more things would be needed for the view metadata builder, which I mentioned above. `TestViewMetadata´ gives a good overview over how the view metadata / builder should behave.

Thanks for working on this btw 💯

Copy link
Collaborator Author

@c-thiel c-thiel Jul 13, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, thanks Eduard for the Feedback.

My preferred way of going forward would be:

  1. Merge this PR which just contains the structs & (de)serialization (very much like the TableMetadaData in its current state)
  2. Figure out a good way for the TableMetadataBuilder including partition binding. We have a first shot ready that we can create a PR for.
  3. Use the same pattern we use for the TableMetadataBuilder for the much lighter ViewMetadataBuilder which then includes all the tests Eduard mentioned.

I am aware that a lot of things are missing in the builder. It was a deliberate decision from me to get views up-to-speed with tables first, and then in a second step extend both tables and views features to the java level. This way we have a handleable PR here that just takes care or (ser)de and can build on it in a next step.

@ZENOTME , @nastra would that be OK for you?

@c-thiel
Copy link
Collaborator Author

c-thiel commented Jul 2, 2024

@nastra, @Fokko during testing we found a Problem with the "default-namespace". I am hoping for some insights from your side:
According to the iceberg view spec, "default-namespace" is required: https://iceberg.apache.org/view-spec/#versions.
As such it is modeled as NamespaceIdent:
https://github.com/c-thiel/iceberg-rust/blob/d8a7aabce73888e5b22712d60ca95b856db66fc2/crates/iceberg/src/spec/view_version.rs#L56

Valid Namespace Identifiers have at least one element, which is validated by the NamespaceIdent constructor.

When creating views, spark creates Metadata objects that specify the "default-namespace" field but as an empty vec. This is a quite unlucky situation, I am not sure what the desired behavior is:

  1. My feeling is that this is a bug in the Iceberg Spec. [] as "default-namespace" provides the same information as a non-required field (null), as a namespace without an element is unusable. However, views in spark work, which shows that default-namespace is not always required. Thus it should be optional.
  2. It's a bug in the spark implementation and in fact "default-namespace" should always contain at least one element.
  3. They are both right, in which case we would except [] as a valid namespace and must change the rust implementation.
  4. [] is in general not a valid namespace unless it is used as a "default-namespace" in which some special behavior is expected. This however brings me back to 1 - why not just use null in this case?

@nastra
Copy link
Contributor

nastra commented Jul 3, 2024

When creating views, spark creates Metadata objects that specify the "default-namespace" field but as an empty vec

Is this with plain OSS Spark or are you using the iceberg-spark-runtime.jar from Iceberg? Just asking, because we're properly handling the namespace when creating a view in https://github.com/apache/iceberg/blob/2a39af894f4f00aa37922ef765cc2583517fa1d1/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala#L51. This code then calls https://github.com/apache/iceberg/blob/81b3310ab469408022cc14af51257b7e8b36614f/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java#L593.

It would ne helpful to understand which Spark + Iceberg version you're using and how you're creating the view in Spark.

@twuebi
Copy link
Contributor

twuebi commented Jul 3, 2024

Hi @nastra,

this script:

import pyspark
from pyspark.conf import SparkConf
from pyspark.sql import SparkSession
import pandas as pd

CATALOG_URL = "http://server:8080/catalog"
MANAGEMENT_URL = "http://server:8080/management"
DEMO_WAREHOUSE = "demo"

config = {
    "spark.sql.catalog.demo-catalog": "org.apache.iceberg.spark.SparkCatalog",
    "spark.sql.catalog.demo-catalog.type": "rest",
    "spark.sql.catalog.demo-catalog.uri": CATALOG_URL,
    "spark.sql.catalog.demo-catalog.warehouse": DEMO_WAREHOUSE,
    "spark.sql.catalog.demo-catalog.io-impl": "org.apache.iceberg.aws.s3.S3FileIO",
    "spark.sql.extensions": "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions",
    "spark.sql.defaultCatalog": "demo-catalog",
    "spark.jars.packages": "org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.5.2,org.apache.iceberg:iceberg-aws-bundle:1.5.2",
}

spark_config = SparkConf().setMaster('local').setAppName("Iceberg-REST")
for k, v in config.items():
    spark_config = spark_config.set(k, v)

spark = SparkSession.builder.config(conf=spark_config).getOrCreate()
spark.sql("CREATE NAMESPACE IF NOT EXISTS spark_demo")
data = pd.DataFrame([[1, 'a-string', 2.2]], columns=['id', 'strings', 'floats'])
sdf = spark.createDataFrame(data)
sdf.writeTo("spark_demo.my_table").createOrReplace()
spark.sql("CREATE view spark_demo.vv as select * from spark_demo.my_table")

ends up sending that POST to /catalog/v1/df910b7e-3912-11ef-a048-6b2680efd54d/namespaces/spark_demo/views

{
  "name": "vv",
  "schema": {
    "schema-id": 0,
    "type": "struct",
    "fields": [
      {
        "id": 0,
        "name": "id",
        "required": false,
        "type": "long"
      },
      {
        "id": 1,
        "name": "strings",
        "required": false,
        "type": "string"
      },
      {
        "id": 2,
        "name": "floats",
        "required": false,
        "type": "double"
      }
    ]
  },
  "view-version": {
    "version-id": 1,
    "schema-id": 0,
    "timestamp-ms": 1719994069884,
    "summary": {
      "app-id": "local-1719994018459",
      "engine-name": "spark",
      "iceberg-version": "Apache Iceberg 1.5.2 (commit cbb853073e681b4075d7c8707610dceecbee3a82)",
      "engine-version": "3.5.1"
    },
    "representations": [
      {
        "type": "sql",
        "sql": "select * from spark_demo.my_table",
        "dialect": "spark"
      }
    ],
    "default-namespace": []
  },
  "properties": {
    "create_engine_version": "Spark 3.5.1",
    "spark.query-column-names": "id,strings,floats",
    "engine_version": "Spark 3.5.1"
  }
}

@twuebi
Copy link
Contributor

twuebi commented Jul 3, 2024

Setting the namespace via:

spark.sql("USE spark_demo")
spark.sql("CREATE view vv3 as select * from spark_demo.mytable")

ends up setting default-namespace:

{
  "name": "vv3",
  "schema": {
    "schema-id": 0,
    "type": "struct",
    "fields": [
      {
        "id": 0,
        "name": "id",
        "required": false,
        "type": "long"
      },
      {
        "id": 1,
        "name": "strings",
        "required": false,
        "type": "string"
      },
      {
        "id": 2,
        "name": "floats",
        "required": false,
        "type": "double"
      }
    ]
  },
  "view-version": {
    "version-id": 1,
    "schema-id": 0,
    "timestamp-ms": 1719994988301,
    "summary": {
      "engine-version": "3.5.1",
      "app-id": "local-1719994018459",
      "engine-name": "spark",
      "iceberg-version": "Apache Iceberg 1.5.2 (commit cbb853073e681b4075d7c8707610dceecbee3a82)"
    },
    "representations": [
      {
        "type": "sql",
        "sql": "select * from spark_demo.my_table",
        "dialect": "spark"
      }
    ],
    "default-namespace": [
      "spark_demo"
    ]
  },
  "properties": {
    "spark.query-column-names": "id,strings,floats",
    "engine_version": "Spark 3.5.1",
    "create_engine_version": "Spark 3.5.1"
  }
}

@nastra
Copy link
Contributor

nastra commented Jul 3, 2024

"default-namespace": [] indicates an empty namespace, not a null one, which is a valid case and there are tests for that here: https://github.com/apache/iceberg/blob/42a2c19cec31c626cbff6cc2dfafb86cdf223bd0/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java#L56. I've also opened apache/iceberg#9890 a while ago to properly test empty namespaces with catalogs that support it.

However, why you're getting an empty namespace when creating the view with a qualified namespace is a different thing (which I need to look into and reproduce on my end - which I'll try to do before the end of the week)

@nastra
Copy link
Contributor

nastra commented Jul 3, 2024

Ok I checked the surrounding code and the handling is correct. In https://github.com/apache/iceberg/blob/2a39af894f4f00aa37922ef765cc2583517fa1d1/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala#L51 we're always determining the current namespace (which can be empty), which is then what ViewVersion#defaultNamespace is set to. As I mentioned earlier that there's a difference between a null namespace and an empty one.

Spark later uses this info in https://github.com/apache/iceberg/blob/6bbf70a52ebccfaba4e7e08facd72b84b571e2a6/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkView.java#L75 in case there's no namespace in the underlying SQL and in your case you fully-qualified select * from spark_demo.my_table with a namespace.

In your second example you configured the current namespace via USE <namespace>, which was then evaluated https://github.com/apache/iceberg/blob/2a39af894f4f00aa37922ef765cc2583517fa1d1/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala#L51 and passed to ViewVersion#defaultNamespace (but not used when actually querying the view).

@twuebi
Copy link
Contributor

twuebi commented Jul 4, 2024

Thanks for checking this @nastra, I understand that we should be relaxing the constraints of NamespaceIdent then since an empty default-namespace is expected.

pub(crate) versions: HashMap<i64, ViewVersionRef>,
/// A list of version log entries with the timestamp and version-id for every
/// change to current-version-id
pub(crate) version_log: Vec<ViewVersionLog>,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

in java we call this history

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hm, in the iceberg spec its called versions & version-log https://iceberg.apache.org/view-spec/#view-metadata.
I implemented the same way TableMetadata is currently implemented: Calling the struct field identically to the spec, but then creating a history accessor.

TableMetadata:
https://github.com/c-thiel/iceberg-rust/blob/ca9de89ac9d95683c8fe9191f72ab922dc4c7672/crates/iceberg/src/spec/table_metadata.rs#L208-L211

ViewMetadata:
https://github.com/c-thiel/iceberg-rust/blob/44630160be1bcf48249c31006b76a7150a029619/crates/iceberg/src/spec/view_metadata.rs#L151-L156

As the versions field is not public, it actually implements almost the same interface as java.

@ZENOTME or some other rust dev, it would be great to get some opinion on this.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I prefer to use similar name as spec as it makes things cleaner. history maybe an alias, and we can add a method for it if required in future.


/// Append view version to view
pub fn append_version(&mut self, view_version: ViewVersion) {
self.current_version_id = view_version.version_id();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I feel that things like view version id re-assignment and view version de-duplication are missing in the builder.
The same applies for schema id re-assignment / schema de-duplication.

There are a bunch of tests in the java ref impl in TestViewMetadata that make sure these things behave correctly

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, you are right. Bad idea to implement this half-done.
I am removing this function and handle it properly later in the full implementation of the builders.

#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)]
#[serde(rename_all = "kebab-case")]
/// A log of when each snapshot was made.
pub struct ViewVersionLog {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

in java this is called ViewHistoryEntry, maybe worth naming it here similarly?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am also happy with renaming.
I think we should use the same naming we settle on in our discussion above. #331 (comment)

}

/// A list of view representations.
pub type ViewRepresentations = Vec<ViewRepresentation>;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is it normal in rust to create a separate type for a list of X?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Normally not. This is looking forward to the ViewMetadataBuilder where we want to implement methods on those.
The motivation is that multiple ViewRepresentations are more than just a Vec - there are many rules in constructing it (i.e. no duplicate dialects). We shouldn't just expose the Vec interface externally because then users could just add an item to the list without going through any of our validation functions (which are not there yet, will come with the ViewMetadataBuilder).

Its a very valuable comment though, because I am still exposing the interface currently. I am putting it in a struct with a private field.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.


/// Add a representation to the list.
///
/// SQL representations dialects must be unique. If a representation with the same
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

does this make sure to ignore the case of the dialect when comparing two dialects? If no, then it should

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, it did not. I fixed it here:
c-thiel@bf95eb7

The behavior is now to ignore the case when adding, but storing the proper case internally (see the test in my commit).
Is this the desired behavior or should we just lowercase it?

/// Add a SQL representation to the list.
///
/// SQL representations dialects must be unique. If a representation with the same
/// dialect already exists, it will be overwritten.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why is it better to overwrite a dialect rather than throw an error saying that there are duplicate dialects?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I guess it depends on the usage. This method is part of a builder which is not only used to create an object initially, but also to modify an object.
If we fail, there is no way to ever modify a dialect in the builder.
I am adding fallible methods additionally in c-thiel@83e4a6f (tests in next commit)

@@ -0,0 +1,58 @@
{
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

However, there isn't much in https://github.com/apache/iceberg/tree/5ea78e3fbe5d8c9c846a1b86bcd2b77d13a31acd/core/src/test/java/org/apache/iceberg/view we can use for now.

In the java code we typically don't construct the full JSON just to test something, but there are plenty of tests in TestViewMetadata to make sure schema / view version re-assignment / de-duplication happens or SQL dialect deduplication behaves correctly.

I just want to start with the foundation before heading into the more controversial builder bits.

I was merely asked to look at this view implementation since I did the Java implementation, but I'm not involved in iceberg-rust, so I'll let others decide how to proceed with adding views to iceberg-rust and what to include in this PR vs in follow-ups.
Regarding follow-ups, I think a few more things would be needed for the view metadata builder, which I mentioned above. `TestViewMetadata´ gives a good overview over how the view metadata / builder should behave.

Thanks for working on this btw 💯

@c-thiel
Copy link
Collaborator Author

c-thiel commented Jul 13, 2024

@nastra I hope I addressed all your issues regarding functionality. If you are happy with the implementations, feel free to close our discussions :).
@ZENOTME it would now be great to get some Feedback on the rust side of things. Please check my discussions with Eduard above. There are a few open points regarding naming and also the introduction of types. It would be great to get a review from you or some other Rust Maintainer for this.

@c-thiel
Copy link
Collaborator Author

c-thiel commented Jul 17, 2024

@nastra just following up on my last comment - could you close our discussion where you agree with my fix?
@liurenjie1024 could you maybe have a look at the rust side of things? Please check my discussions with Eduard above. There are a few open points regarding naming and also the introduction of types. It would be great to get a review from you or some other Rust Maintainer for this.

@liurenjie1024
Copy link
Contributor

@nastra just following up on my last comment - could you close our discussion where you agree with my fix? @liurenjie1024 could you maybe have a look at the rust side of things? Please check my discussions with Eduard above. There are a few open points regarding naming and also the introduction of types. It would be great to get a review from you or some other Rust Maintainer for this.

Sure, I'll find time this week to go throught this.

Copy link
Contributor

@liurenjie1024 liurenjie1024 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @c-thiel for this pr, it looks great to me! I've left some minor suggestion and some question.

self.0.into_iter()
}
}
impl<'a> IntoIterator for &'a ViewRepresentations {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why not impl Iterator<Item=&ViewRepresentation>?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh, I screwed up the impl, we cannot implement Iterator on an immutable object - no place to store the state. However, I just implemented iter() on the object itself.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

}

/// A builder for [`ViewRepresentations`].
pub struct ViewRepresentationsBuilder(Vec<ViewRepresentation>);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we remove this part for now? I'm expecting view manipulation would be similar to table transaction ?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Don't we still need it to create an action?

Should we keep the ViewMetadataBuilder?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Don't we still need it to create an action?

Yes, I think we will eventually need it, but I don't think we should expose it now as a public api. What I mean is to add it later when we actually start to implement view related action.

Should we keep the ViewMetadataBuilder?

I think we should keep it, it's useful for simple case, like creating a new view.

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, I removed the builder in c-thiel@06d09ab.
Is there an issue already for the transaction we could mention this commit in? I assume most of the tests and maybe a bit of the logic might be re-usable.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there an issue already for the transaction we could mention this commit in? I assume most of the tests and maybe a bit of the logic might be re-usable.

Not yet. Feel free to create issues to track them. Recently I've been focusing on table read part and 0.3 release, but never hesitate to create issues if you are interested in view support.

pub(crate) versions: HashMap<i64, ViewVersionRef>,
/// A list of version log entries with the timestamp and version-id for every
/// change to current-version-id
pub(crate) version_log: Vec<ViewVersionLog>,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I prefer to use similar name as spec as it makes things cleaner. history maybe an alias, and we can add a method for it if required in future.

}

/// Returns the last updated timestamp as a DateTime<Utc> with millisecond precision.
pub fn timestamp(self) -> MappedLocalTime<DateTime<Utc>> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why we return MappedLocalTime here? Why not just DateTime<Utc> directly?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Conversion to DateTime<Utc> is fallible - thus it would become a Result. We already have timestamp_ms() as a non-fallible version. Do you still think we should change?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What I absolutely don't like is the TableMetadata way of doing this which just panics:

Utc.timestamp_millis_opt(self.last_updated_ms).unwrap()

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for explaination, I didn't notice that chrono changed their api. To be hones, I feel the name MappedLocalTime is a little difficult to understand. Also the doc says that it's only Single or None. I agree that we should not use unwrap in table_metadata. How about we convert it to crate::error::Result, e.g.

Single => Ok,
Ambiguous => Err(Unexpected),
None => Err(InvalidData)

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Collaborator Author

@c-thiel c-thiel Jul 23, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

And my proposal for TableMetadata.
c-thiel@1c49ff9

Do you have a better Idea for method naming? I think last_updated_ms was a really bad method name for returning a DateTime in the first place. Nonetheless, it now changes the behavior to return an int instead so that we still have a non-fallible accessor.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The name last_updated_timestamp is good engouh for me. Thanks for changing this.

/// This is a helper module that defines types to help with serialization/deserialization.
/// For deserialization the input first gets read into either the [SnapshotV1] or [SnapshotV2] struct
/// and then converted into the [Snapshot] struct. Serialization works the other way around.
/// [SnapshotV1] and [SnapshotV2] are internal struct that are only used for serialization and deserialization.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems this is not change?

}

/// A builder for [`ViewRepresentations`].
pub struct ViewRepresentationsBuilder(Vec<ViewRepresentation>);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Don't we still need it to create an action?

Yes, I think we will eventually need it, but I don't think we should expose it now as a public api. What I mean is to add it later when we actually start to implement view related action.

Should we keep the ViewMetadataBuilder?

I think we should keep it, it's useful for simple case, like creating a new view.

default_namespace,
summary,
} = view_creation;
let initial_version_id = super::INITIAL_SEQUENCE_NUMBER;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should be 1?

Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You are right. In Java it seems to be one as well.
Why though? SortOrder, Schema, PartitionSpec all start from 0. Why not views?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the concept of a sequence number is thing that's only used for tables and not for views, so using INITIAL_SEQUENCE_NUMBERseems rather confusing. However, I don't think we actually validate that the view version starts at 1 I think. Other places that create a new view version however always start at 1, such as https://github.com/apache/iceberg/blob/63af974efe51486c89bff8df5416781ab3181976/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java#L1275

Copy link
Collaborator Author

@c-thiel c-thiel Jul 24, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Lets stick to 1 then :)

Fixed in 792129d

}

/// Returns the last updated timestamp as a DateTime<Utc> with millisecond precision.
pub fn timestamp(self) -> MappedLocalTime<DateTime<Utc>> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for explaination, I didn't notice that chrono changed their api. To be hones, I feel the name MappedLocalTime is a little difficult to understand. Also the doc says that it's only Single or None. I agree that we should not use unwrap in table_metadata. How about we convert it to crate::error::Result, e.g.

Single => Ok,
Ambiguous => Err(Unexpected),
None => Err(InvalidData)

Copy link
Contributor

@liurenjie1024 liurenjie1024 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @c-thiel for this high quality pr, and I think we are quite close to merge it. I have several minor nits to fix, but in general LGTM.

pub fn timestamp(&self) -> MappedLocalTime<DateTime<Utc>> {
Utc.timestamp_millis_opt(self.timestamp_ms)
pub fn timestamp(&self) -> Result<DateTime<Utc>> {
match Utc.timestamp_millis_opt(self.timestamp_ms) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This appears several times, maybe we can move it to error module?

Copy link
Collaborator Author

@c-thiel c-thiel Jul 25, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hm, I added it to the error module but I am not happy with it.
c-thiel@82dc949

The problem is that the return value of Utc.timestamp_millis_opt(timestamp_ms) offers no built-in method to convert it to an Result. So we cannot use ? but require a uitility function instead. The one I've written is only remotely related to errors - which is why I don't really like it to be in errors.rs. I am not sure though if it's worth adding a new utility module just for timestamp conversions.

Any thoughts?

I made it crate internal - so maybe its not too bad.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the MappedLocalTime is somehow another kind of result with extra values, so putting it in error module seems fine to me. Also it's not a public api, so we wil be able to change it in future if we have better solutions.

}

/// Returns the last updated timestamp as a DateTime<Utc> with millisecond precision.
pub fn timestamp(self) -> MappedLocalTime<DateTime<Utc>> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The name last_updated_timestamp is good engouh for me. Thanks for changing this.

Copy link
Contributor

@liurenjie1024 liurenjie1024 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @c-thiel 's effort on this, looks great to me!

@liurenjie1024
Copy link
Contributor

cc @Xuanwo @nastra @Fokko to take another look

@c-thiel
Copy link
Collaborator Author

c-thiel commented Jul 27, 2024

Rebased on current master

@liurenjie1024
Copy link
Contributor

I think this pr is good enough, so I will merge it since it has been long. Thanks @c-thiel 's effort on this, and thanks @Xuanwo @nastra @ZENOTME 's review!

@liurenjie1024 liurenjie1024 merged commit ab4f69a into apache:main Jul 28, 2024
11 checks passed
github-merge-queue bot pushed a commit to risingwavelabs/iceberg-rust that referenced this pull request Sep 9, 2024
* feat: Add website layout (#130)

* feat: Add website layout

Signed-off-by: Xuanwo <[email protected]>

* publish to rust.i.a.o

Signed-off-by: Xuanwo <[email protected]>

* Fix license

Signed-off-by: Xuanwo <[email protected]>

* Let's try mdbook action

Signed-off-by: Xuanwo <[email protected]>

* use cargo install

Signed-off-by: Xuanwo <[email protected]>

* disable section

Signed-off-by: Xuanwo <[email protected]>

* Add docs for website

Signed-off-by: Xuanwo <[email protected]>

* Fix license

Signed-off-by: Xuanwo <[email protected]>

* action approved

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* feat: Expression system. (#132)

* feat: Expressions

* Fix comments

* Refactor expression to be more similar to iceberg model

* Fix typo

* website: Fix typo in book.toml (#136)

Signed-off-by: Xuanwo <[email protected]>

* Set ghp_path and ghp_branch properties (#138)

* chore: Upgrade toolchain to 1.75.0 (#140)

* feat: Add roadmap and features status in README.md (#134)

* feat: Add roadmap and features status in README.md

* Fix

* Fix

* Add more details according to comments

* Revert unnecessary new line break

* Nits

---------

Co-authored-by: Fokko Driesprong <[email protected]>

* Infra: Remove `publish:` section from `.asf.yaml` (#141)

* chore(deps): Bump peaceiris/actions-gh-pages from 3.9.2 to 3.9.3 (#143)

Bumps [peaceiris/actions-gh-pages](https://github.com/peaceiris/actions-gh-pages) from 3.9.2 to 3.9.3.
- [Release notes](https://github.com/peaceiris/actions-gh-pages/releases)
- [Changelog](https://github.com/peaceiris/actions-gh-pages/blob/main/CHANGELOG.md)
- [Commits](https://github.com/peaceiris/actions-gh-pages/compare/v3.9.2...v3.9.3)

---
updated-dependencies:
- dependency-name: peaceiris/actions-gh-pages
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* chore(deps): Update opendal requirement from 0.43 to 0.44 (#142)

Updates the requirements on [opendal](https://github.com/apache/incubator-opendal) to permit the latest version.
- [Release notes](https://github.com/apache/incubator-opendal/releases)
- [Changelog](https://github.com/apache/incubator-opendal/blob/main/CHANGELOG.md)
- [Commits](https://github.com/apache/incubator-opendal/compare/v0.43.0...v0.43.0)

---
updated-dependencies:
- dependency-name: opendal
  dependency-type: direct:production
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* docs: Change homepage to rust.i.a.o (#146)

* feat: Introduce basic file scan planning. (#129)

* Code complete

* Resolve

* Done

* Fix comments

* Fix comments

* Fix comments

* Fix

* Fix comment

* chore: Update contributing guide. (#163)

* chore: Update reader api status (#162)

* chore: Update reader api status

* Restore unnecessary change

* #154 : Add homepage to Cargo.toml (#160)

* Add formatting for toml files (#167)

* Add formatting for toml files

* Update call to taplo

* Add command to format and a command to check

* chore(deps): Update env_logger requirement from 0.10.0 to 0.11.0 (#170)

Updates the requirements on [env_logger](https://github.com/rust-cli/env_logger) to permit the latest version.
- [Release notes](https://github.com/rust-cli/env_logger/releases)
- [Changelog](https://github.com/rust-cli/env_logger/blob/main/CHANGELOG.md)
- [Commits](https://github.com/rust-cli/env_logger/compare/v0.10.0...v0.10.2)

---
updated-dependencies:
- dependency-name: env_logger
  dependency-type: direct:production
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* feat: init file writer interface (#168)

* init file writer interface

* refine

---------

Co-authored-by: ZENOTME <[email protected]>

* fix: Manifest parsing should consider schema evolution. (#171)

* fix: Manifest parsing should consider schema evolution.

* Fix ut

* docs: Add release guide for iceberg-rust (#147)

* fix: Ignore negative statistics value (#173)

* feat: Add user guide for website. (#178)

* Add

* Fix format

* Add license header

* chore(deps): Update derive_builder requirement from 0.12.0 to 0.13.0 (#175)

Updates the requirements on [derive_builder](https://github.com/colin-kiegel/rust-derive-builder) to permit the latest version.
- [Release notes](https://github.com/colin-kiegel/rust-derive-builder/releases)
- [Commits](https://github.com/colin-kiegel/rust-derive-builder/compare/v0.12.0...v0.12.0)

---
updated-dependencies:
- dependency-name: derive_builder
  dependency-type: direct:production
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Replace unwrap (#183)

* feat: add handwritten serialize (#185)

* add handwritten serialize

* revert expect

* remove expect

* Fix avro schema names for manifest and manifest_list (#182)

Co-authored-by: Fokko Driesprong <[email protected]>

* feat: Bump hive_metastore to use pure rust thrift impl `volo` (#174)

* feat: Bump version 0.2.0 to prepare for release. (#181)

* feat: Bump version 0.2.0 to prepare for release.

* Update dependencies

* fix: `default_partition_spec` using the `partion_spec_id` set (#190)

* add unit tests

* fix type

* Docs: Add required Cargo version to install guide (#191)

* chore(deps): Update opendal requirement from 0.44 to 0.45 (#195)

Updates the requirements on [opendal](https://github.com/apache/opendal) to permit the latest version.
- [Release notes](https://github.com/apache/opendal/releases)
- [Changelog](https://github.com/apache/opendal/blob/main/CHANGELOG.md)
- [Commits](https://github.com/apache/opendal/compare/v0.44.0...v0.44.2)

---
updated-dependencies:
- dependency-name: opendal
  dependency-type: direct:production
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Smooth out release steps (#197)

Couple of small things:

- The license check failed because the `dist/*` files were there
- Add `dist/*` to gitignore since we don't want to push these files to the repo
- Make `scripts/release.sh` executable
- Align the svn structure with PyIceberg and Java

* refactor: remove support of manifest list format as a list of file path (#201)

* refactor: remove support of manifest list format as a list of file paths#158

* refactor: add field definition to manifest list

* refactor: delete duplicated function

* refactor: fix duplicate function name

* refactor: remove unwraps (#196)

* remove avro unwraps

* rm unwrap in schema manifest

* rm some expects

* rm types

* fix clippy

* fix string format

* refine some unwrap

* undo schema.rs

* Fix: add required rust version in cargo.toml (#193)

* Fix: add required rust version in cargo.toml

* added rust-version to workspace=true in package

* Fix the REST spec version (#198)

This number indicates from which release the code was generated.
For example, currently new endpoints are added to the spec, but
they are not supported by iceberg-rust yet.

* feat: Add Sync + Send to Catalog trait (#202)

* feat: Make thrift transport configurable (#194)

* feat: make transport configurable (#188)

* implement default for HmsThriftTransport

* Add UnboundSortOrder (#115)

* Add UnboundSortOrder

* Separate build methods for bound and unbound

* Use a constant for unsorted order_id

* ci: Add workflow for publish (#218)

* ci: Add workflow for publish

Signed-off-by: Xuanwo <[email protected]>

* Fix publish

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* ci: add workflow for cargo audit (#217)

* docs: Add basic README for all crates (#215)

* docs: Add basic README for all crates

Signed-off-by: Xuanwo <[email protected]>

* Remove license

Signed-off-by: Xuanwo <[email protected]>

* Update links

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* Follow naming convention from Iceberg's Java and Python implementations (#204)

* doc: Add download page (#219)

* doc: Add download page

* Fix links

* chore(deps): Update derive_builder requirement from 0.13.0 to 0.20.0 (#203)

Updates the requirements on [derive_builder](https://github.com/colin-kiegel/rust-derive-builder) to permit the latest version.
- [Release notes](https://github.com/colin-kiegel/rust-derive-builder/releases)
- [Commits](https://github.com/colin-kiegel/rust-derive-builder/compare/v0.13.0...v0.13.1)

---
updated-dependencies:
- dependency-name: derive_builder
  dependency-type: direct:production
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* test: add FileIO s3 test (#220)

* add file io s3 test

* add license

* fixed version & rm port scanner

* ci: Ignore RUSTSEC-2023-0071 for no actions to take (#222)

* ci: Ignore RUSTSEC-2023-0071 for no actions to take

Signed-off-by: Xuanwo <[email protected]>

* Fix license header

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* feat: Add expression builder and display. (#169)

* feat: Add expression builder and display.

* Fix comments

* Fix doc test

* Fix name of op

* Fix comments

* Fix timestamp

* chord:  Add IssueNavigationLink for RustRover (#230)

* chord:  IssueNavigationLink for RustRover

* move to .idea

* add apache license

---------

Co-authored-by: fuqijun <[email protected]>

* minor: Fix `double` API doc typo (#226)

* feat: add `UnboundPredicate::negate()` (#228)

Issue: #150

* fix: Remove deprecated methods to pass ci (#234)

* Implement basic Parquet data file reading capability (#207)

* feat: TableScan parquet file read to RecordBatch stream

* chore: add inline hinting and fix incorrect comment

* refactor: extract record batch reader

* refactor: rename `FileRecordBatchReader` to `ArrowReader`

* refactor: rename file_record_batch_reader.rs to arrow.rs
* refactor: move `batch_size` param to `TableScanBuilder`
* refactor: rename `TableScan.execute` to `to_arrow`

* refactor: use builder pattern to create `ArrowReader`

* chore: doc-test as a target (#235)

* feat: add parquet writer (#176)

* Add hive metastore catalog support (part 1/2) (#237)

* fmt members

* setup basic test-infra for hms-catalog

* add license

* add hms create_namespace

* add hms get_namespace

* fix: typo

* add hms namespace_exists and drop_namespace

* add hms update_namespace

* move fns into HmsCatalog

* use `expose` in docker-compose

* add hms list_tables

* fix: clippy

* fix: cargo sort

* fix: cargo workspace

* move fns into utils + add constants

* include database name in error msg

* add pilota to cargo workspace

* add minio version

* change visibility to pub(crate); return namespace from conversion fn

* add minio version in rest-catalog docker-compose

* fix: hms test docker infrastructure

* add version to minio/mc

* fix: license header

* fix: core-site

---------

Co-authored-by: mlanhenke <[email protected]>

* chore: Enable projects. (#247)

* Make plan_files as asynchronous stream (#243)

* feat: Implement binding expression (#231)

* feat: Implement binding expression

* Implement Display instead of ToString (#256)

* add rewrite_not (#263)

* feat: init TableMetadataBuilder (#262)

* Rename stat_table to table_exists in Catalog trait (#257)

* feat (static table): implement a read-only table struct loaded from metadata (#259)

* fixing some broken branch

* adding readonly property to Table, and setting readonly value on StaticTable

* feat: implement OAuth for catalog rest client (#254)

* docs: annotate precision and length to primitive types (#270)

Signed-off-by: Ruihang Xia <[email protected]>

* build: Restore CI by making parquet and arrow version consistent (#280)

* Metadata Serde + default partition_specs and sort_orders (#272)

* change serde metadata v2

* change default partition_specs and sort_orders

* change test

* use DEFAULTS

* feat: make optional oauth param configurable (#278)

* make optional oauth param configurable

* fix review comments.

---------

Co-authored-by: hpal <[email protected]>

* fix: enable public access to ManifestEntry properties (#284)

* enable public access to ManifestEntry properties

* implementing getter methods instead of direct access

* feat: Implement the conversion from Arrow Schema to Iceberg Schema (#258)

* feat: Implement the conversion from ArrowSchema to iceberg Schema

* For review

* Update test

* Add LargeString, LargeBinary, LargeList and FixedSizeList

* Add decimal type

* For review

* Fix clippy

* Rename funtion name to add_manifests (#293)

* feat: modify `Bind` calls so that they don't consume `self` and instead return a new struct, leaving the original unmoved" (#290)

* Add hive metastore catalog support (part 2/2) (#285)

* fmt members

* setup basic test-infra for hms-catalog

* add license

* add hms create_namespace

* add hms get_namespace

* fix: typo

* add hms namespace_exists and drop_namespace

* add hms update_namespace

* move fns into HmsCatalog

* use `expose` in docker-compose

* add hms list_tables

* fix: clippy

* fix: cargo sort

* fix: cargo workspace

* move fns into utils + add constants

* include database name in error msg

* add pilota to cargo workspace

* add minio version

* change visibility to pub(crate); return namespace from conversion fn

* add minio version in rest-catalog docker-compose

* fix: hms test docker infrastructure

* add version to minio/mc

* fix: license header

* fix: core-site

* split utils and errors

* add fn get_default_table_location

* add fn get_metadata_location

* add docs

* add HiveSchemaBuilder

* add schema to HiveSchemaBuilder

* add convert_to_hive_table

* cargo sort

* implement table_ops without TableMetadataBuilder

* refactor: HiveSchema fn from_iceberg

* prepare table creation without metadata

* simplify HiveSchemaBuilder

* refactor: use ok_or_else()

* simplify HiveSchemaBuilder

* fix visibility of consts

* change serde metadata v2

* change default partition_specs and sort_orders

* change test

* add create table with metadata

* use FileIO::from_path

* add test_load_table

* small fixes + docs

* rename

* extract get_metadata_location from hive_table

* add integration tests

* fix: clippy

* remove whitespace

* fix: fixture names

* remove builder-prefix `with`

* capitalize error msg

* remove trait bound `Display`

* add const `OWNER`

* fix: default warehouse location

* add test-case `list_tables`

* add all primitives to test_schema

* exclude `Timestamptz` from hive conversion

* remove Self::T from schema

* remove context

* keep file_io in HmsCatalog

* use json schema repr

---------

Co-authored-by: mlanhenke <[email protected]>

* feat: implement prune column for schema (#261)

* feat: implement PruneColumn for Schema

* fix: fix bugs for PruneColumn implementation

* test: add test cases for PruneColumn

* fix: fix minor to make more rusty

* fix: fix cargo clippy

* fix: construct expected_type from SchemaBuilder

* fix: more readability

* change return type of prune_column

* chore(deps): Update reqwest requirement from ^0.11 to ^0.12 (#296)

* Glue Catalog: Basic Setup + Test Infra (1/3) (#294)

* extend dependency DIRS

* create dependencies for glue

* basic setup

* rename test

* add utils/get_sdk_config

* add tests

* add list_namespace

* fix: clippy

* fix: unused

* fix: workspace

* fix: name

* use creds in test-setup

* fix: empty dependencies.rust.tsv

* fix: rename endpoint_url

* remove deps.tsv

* add hms deps.tsv

* fix deps.tsv

* fix: deps.tsv

* feat: rest client respect prefix prop (#297)

* feat: rest client respect prefix prop

Signed-off-by: TennyZhuang <[email protected]>

* add test

Signed-off-by: TennyZhuang <[email protected]>

* fix tests without prefix

Signed-off-by: TennyZhuang <[email protected]>

* fix clippy

Signed-off-by: TennyZhuang <[email protected]>

---------

Signed-off-by: TennyZhuang <[email protected]>

* fix: missing properties (#303)

* fix: renaming FileScanTask.data_file to data_manifest_entry (#300)

* renaming FileScanTask.data_file to data_manifest_entry

* renaming data_file.content() to content_type()

* changing pub method to data()

* feat: Make OAuth token server configurable (#305)

* feat: Glue Catalog - namespace operations (2/3) (#304)

* add from_build_error

* impl create_namespace

* impl get_namespace

* add macro with_catalog_id

* impl namespace_exists

* impl update_namespace

* impl list_tables

* impl drop_namespace

* fix: clippy

* update docs

* update docs

* fix: naming and visibility of error conversions

* feat: add transform_literal (#287)

* add transform_literal

* refine

* fix unwrap

---------

Co-authored-by: ZENOTME <[email protected]>

* feat: Complete predicate builders for all operators. (#276)

* feat: Complete predicate builders for all operators.

* ci: fix fmt error

* fix nan and notnan

* feat: Support customized header in Rest catalog client (#306)

Note that: the default headers will not be overwritten.

* fix: chrono dep (#274)

* feat: Read Parquet data file with projection (#245)

* feat: Read Parquet data file with projection

* fix

* Update

* More

* For review

* Use FeatureUnsupported error.

* Fix day timestamp micro (#312)

* basic fix

* change to Result<i32>

* use try_unary

* feat: support uri redirect in rest client (#310)

Signed-off-by: TennyZhuang <[email protected]>

* refine: seperate parquet reader and arrow convert (#313)

* Upgrade to rust-version 1.77.1 (#316)

* Support identifier warehouses (#308)

* Support identifier warehouses

This is a bit confusing if you come from a Hive background
where the warehouse is always a path to hdfs/s3/etc.

With the REST catalog, the warehouse can also be a logical
identifier:
https://github.com/apache/iceberg/blob/main/open-api/rest-catalog-open-api.yaml#L72-L78

This means that we have to make sure that we only parse paths
that are an actual path, and not an identifier.

I'm open to suggestions. The check is now very simple, but can
be extended for example using a regex. But I'm not sure what
the implications are of importing additional packages (in Python
you want to keep it as lightweight as possible).

* Use `if Url::parse().is_ok()`

* feat: Project transform (#309)

* add project bucket_unary

* add project bucket_binary

* add project bucket_set

* add project identity

* add project truncate

* fixed array boundary

* add project void

* add project unknown

* add docs + none projections

* docs

* docs

* remove trait + impl boundary on Datum

* fix: clippy

* fix: test Transform::Unknown

* add: transform_literal_result

* add: transform_literal_result

* remove: whitespace

* move `boundary` to transform.rs

* add check if transform can be applied to data_type

* add check

* add: java-testsuite Transform::Bucket

* fix: clippy

* add: timestamps to boundary

* change: return bool from can_transform

* fix: clippy

* refactor: fn project match structure

* add: java-testsuite Transform::Truncate

* add: java-testsuite Transform::Dates + refactor

* fix: doc

* add: timestamp test + refactor

* refactor: simplify projected_boundary

* add: java-testsuite Transform::Timestamp

* refactor tests

* fix: timestamp conversion

* fix: temporal test_result

* basic fix

* change to Result<i32>

* use try_unary

* add: java-testsuite Transform::Timestamp Hours

* refactor: split and move tests

* refactor: move transform tests

* remove self

* refactor: structure fn project + helpers

* fix: clippy

* fix: typo

* fix: naming + generics

* feat: add Struct Accessors to BoundReferences (#317)

* feat: use str args rather than String in transform (#325)

* chore(deps): Update pilota requirement from 0.10.0 to 0.11.0 (#327)

Updates the requirements on [pilota](https://github.com/cloudwego/pilota) to permit the latest version.
- [Release notes](https://github.com/cloudwego/pilota/releases)
- [Commits](https://github.com/cloudwego/pilota/compare/pilota-0.10.0...pilota-0.10.0)

---
updated-dependencies:
- dependency-name: pilota
  dependency-type: direct:production
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* chore(deps): Bump peaceiris/actions-mdbook from 1 to 2 (#332)

Bumps [peaceiris/actions-mdbook](https://github.com/peaceiris/actions-mdbook) from 1 to 2.
- [Release notes](https://github.com/peaceiris/actions-mdbook/releases)
- [Changelog](https://github.com/peaceiris/actions-mdbook/blob/main/CHANGELOG.md)
- [Commits](https://github.com/peaceiris/actions-mdbook/compare/v1...v2)

---
updated-dependencies:
- dependency-name: peaceiris/actions-mdbook
  dependency-type: direct:production
  update-type: version-update:semver-major
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* chore(deps): Bump peaceiris/actions-gh-pages from 3.9.3 to 4.0.0 (#333)

Bumps [peaceiris/actions-gh-pages](https://github.com/peaceiris/actions-gh-pages) from 3.9.3 to 4.0.0.
- [Release notes](https://github.com/peaceiris/actions-gh-pages/releases)
- [Changelog](https://github.com/peaceiris/actions-gh-pages/blob/main/CHANGELOG.md)
- [Commits](https://github.com/peaceiris/actions-gh-pages/compare/v3.9.3...v4.0.0)

---
updated-dependencies:
- dependency-name: peaceiris/actions-gh-pages
  dependency-type: direct:production
  update-type: version-update:semver-major
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* chore(deps): Bump apache/skywalking-eyes from 0.5.0 to 0.6.0 (#328)

Bumps [apache/skywalking-eyes](https://github.com/apache/skywalking-eyes) from 0.5.0 to 0.6.0.
- [Release notes](https://github.com/apache/skywalking-eyes/releases)
- [Changelog](https://github.com/apache/skywalking-eyes/blob/main/CHANGES.md)
- [Commits](https://github.com/apache/skywalking-eyes/compare/v0.5.0...v0.6.0)

---
updated-dependencies:
- dependency-name: apache/skywalking-eyes
  dependency-type: direct:production
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* feat: add BoundPredicateVisitor. Add AlwaysTrue and AlwaysFalse to Predicate (#334)

* feat: add InclusiveProjection (#335)

* feat: Implement the conversion from Iceberg Schema to Arrow Schema (#277)

* support iceberg schema to arrow schema

* avoid realloc hashmap

---------

Co-authored-by: ZENOTME <[email protected]>

* Simplify expression when doing `{and,or}` operations (#339)

This will make sure that we nicely reduce the expression in
the inclusive projection visitor:

https://github.com/apache/iceberg-rust/blob/de80a2436bb2fbbd5b4ec6bcafd0bd041b263595/crates/iceberg/src/expr/visitors/inclusive_projection.rs#L73

* feat: Glue Catalog - table operations (3/3) (#314)

* add GlueSchemaBuilder

* add warehouse

* add serde_json, tokio, uuid

* add minio

* add create_table

* add tests utils

* add load_table

* add drop_table + table_exists

* add rename_table

* add docs

* fix: docs + err_msg

* fix: remove unused const

* fix: default_table_location

* fix: remove single quotes error message

* chore: add test-condition `test_rename_table`

* chore: add test-condition `test_table_exists`

* chore: update roadmap (#336)

* chore: update roadmap

* chore: update reader section

* fix: read into arrow record batch

* feat: add ManifestEvaluator (#322)

* feat: init iceberg writer (#275)

* init iceberg writer

* refine

* refine the interface

---------

Co-authored-by: ZENOTME <[email protected]>

* feat: implement manifest filtering in TableScan (#323)

* Refactor: Extract `partition_filters` from `ManifestEvaluator` (#360)

* refactor: extract inclusive_projection from manifest_evaluator

* refactor: add FileScanStreamContext

* refactor: create partition_spec and partition_schema

* refactor: add cache structs

* refactor: use entry in partition_file_cache

* refactor: use result

* chore: update docs + fmt

* refactor: add bound_filter to FileScanStreamContext

* refactor: return ref BoundPredicate

* fix: return type PartitionSpecRef

* refactor: remove spec_id runtime check

* feat: add check for content_type data

* Basic Integration with Datafusion (#324)

* chore: basic structure

* feat: add IcebergCatalogProvider

* feat: add IcebergSchemaProvider

* feat: add IcebergTableProvider

* chore: add integration test infr

* fix: remove old test

* chore: update crate structure

* fix: remove workspace dep

* refactor: use try_join_all

* chore: remove feature flag

* chore: rename package

* chore: update readme

* feat: add TableType

* fix: import + async_trait

* fix: imports + async_trait

* chore: remove feature flag

* fix: cargo sort

* refactor: CatalogProvider `fn try_new`

* refactor: SchemaProvider `fn try_new`

* chore: update docs

* chore: update docs

* chore: update doc

* feat: impl `fn schema` on TableProvider

* chore: rename ArrowSchema

* refactor: remove DashMap

* feat: add basic IcebergTableScan

* chore: fix docs

* chore: add comments

* fix: clippy

* fix: typo

* fix: license

* chore: update docs

* chore: move derive stmt

* fix: collect into hashmap

* chore: use DFResult

* Update crates/integrations/datafusion/README.md

Co-authored-by: Liang-Chi Hsieh <[email protected]>

---------

Co-authored-by: Renjie Liu <[email protected]>
Co-authored-by: Liang-Chi Hsieh <[email protected]>

* refactor: cache partition_schema in `fn plan_files()` (#362)

* refactor: add partition_schema_cache

* refactor: use context as param object

* fix: test setup

* refactor: clone only when cache miss

* chore: move derive stmts

* refactor: remove unused case_sensitive parameter

* refactor: remove partition_schema_cache

* refactor: move partition_filter into wider scope

* fix (manifest-list): added serde aliases to support both forms conventions (#365)

* added serde aliases to support both forms conventions

* reading manifests without avro schema

* adding avro files of both versions and add a test to deser both

* fixed typo

* feat: Extract FileRead and FileWrite trait (#364)

* feat: Extract FileRead and FileWrie trait

Signed-off-by: Xuanwo <[email protected]>

* Enable s3 services for tests

Signed-off-by: Xuanwo <[email protected]>

* Fix sort

Signed-off-by: Xuanwo <[email protected]>

* Add comment for io trait

Signed-off-by: Xuanwo <[email protected]>

* Fix test for rest

Signed-off-by: Xuanwo <[email protected]>

* Use try join

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* feat: Convert predicate to arrow filter and push down to parquet reader (#295)

* feat: Convert predicate to arrow filter and push down to parquet reader

* For review

* Fix clippy

* Change from vector of BoundPredicate to BoundPredicate

* Add test for CollectFieldIdVisitor

* Return projection_mask for leaf column

* Update

* For review

* For review

* For review

* For review

* More

* fix

* Fix clippy

* More

* Fix clippy

* fix clippy

* chore(deps): Update datafusion requirement from 37.0.0 to 38.0.0 (#369)

* chore(deps): Update itertools requirement from 0.12 to 0.13 (#376)

* Add `InclusiveMetricsEvaluator` (#347)

* feat: add InclusiveMetricsEvaluator

* test: add more tests for InclusiveMetricsEvaluator

* Rename V2 spec names. (#380)

* make file scan task serializable (#377)

Co-authored-by: ZENOTME <[email protected]>

* Feature: Schema into_builder method (#381)

* replaced `i32` in `TableUpdate::SetDefaultSortOrder` to `i64` (#387)

* fix: make PrimitiveLiteral and Literal not be Ord (#386)

* make PrimitiveLiteral and Literal not be Ord

* refine Map

* fix name

* fix map test

* refine

---------

Co-authored-by: ZENOTME <[email protected]>

* docs(writer/docker): fix small typos and wording (#389)

* docs: fixup docker compose test_utils

* docs: iceberg writer close fn

* feat: `StructAccessor.get` returns `Result<Option<Datum>>` instead of `Result<Datum>` (#390)

This is so that the accessor's result can represent null field values.

Fixes: #379

* feat: add `ExpressionEvaluator` (#363)

* refactor: add partition_schema_cache

* refactor: use context as param object

* fix: test setup

* refactor: clone only when cache miss

* chore: move derive stmts

* feat: add basic setup expression evaluator

* refactor: remove unused case_sensitive parameter

* chore: add doc

* refactor: remove partition_schema_cache

* refactor: move partition_filter into wider scope

* feat: add expression_evaluator_cache and apply in scan.rs

* chore: remove comment

* refactor: remove unused test setup fn

* feat: add basic test infr + simple predicate evaluation

* fix: clippy

* feat: impl `is_null` + `not_null`

* feat: impl `is_nan` + `not_nan`

* chore: change result type

* feat: impl `less_than` + `greater_than`

* chore: fix return type

* feat: impl `eq` + `not_eq`

* feat: impl `starts_with + `not_starts_with`

* feat: impl  +

* chore: add tests for and and or expr

* chore: move test

* chore: remove unused_vars

* chore: update docs

* chore: update docs

* fix: typo

* refactor: compare datum instead of primitive literal

* refactor: use Result<Option> from accessor

* chore: remove unused fn

* fix: sdd sleep pattern matching

* Derive Clone for TableUpdate (#402)

* Add accessor for Schema identifier_field_ids (#388)

* Add accessor for Schema identifier_field_ids

* dont expose HashSet

Co-authored-by: Renjie Liu <[email protected]>

* fix

* Fix accessor

---------

Co-authored-by: Renjie Liu <[email protected]>

* deps: Bump arrow related crates to 52 (#403)

* SnapshotRetention::Tag max_ref_age_ms should be optional (#391)

* feat: Add storage features for iceberg (#400)

* feat: Add storage features for iceberg

Signed-off-by: Xuanwo <[email protected]>

* Format toml

Signed-off-by: Xuanwo <[email protected]>

* Add fs and s3 into default features

Signed-off-by: Xuanwo <[email protected]>

* Make toml happy

Signed-off-by: Xuanwo <[email protected]>

* Remove not needed feature flag

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* Implement `BoundPredicateVisitor` for `ManifestFilterVisitor` (#367)

* Implement all functions of BoundPredicateVisitor for ManifestFilterVisitor

* Fix code comments

* Refactor code and fixpredicate for is_some_and

* Refactor code

* Handle review comments

* Handle review comments

* Handle review comments

* Refactor code

* Add missing arrow predicate pushdown implementations for `StartsWith`, `NotStartsWith`, `In`, and `NotIn` (#404)

* feat: add [not_]starts_with and [not_]in arrow predicate pushdown

* fixes from issues highlighted in review

* feat: make BoundPredicate,Datum serializable (#406)

* make BoundPredicate,Datum serializable

* refine error

* fix float check

* use value instead of string to avoid precision loss

---------

Co-authored-by: ZENOTME <[email protected]>

* refactor: Upgrade hive_metastore to 0.1 (#409)

* refactor: Upgrade hive_metastore to 0.1

Signed-off-by: Xuanwo <[email protected]>

* format toml

Signed-off-by: Xuanwo <[email protected]>

* Fix typo

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* Remove duplicate filter (#414)

* Enhancement: refine the reader interface (#401)

* refactor(catalog/rest): Split http client logic to seperate mod (#423)

Signed-off-by: Xuanwo <[email protected]>

* Remove #[allow(dead_code)] from the codebase (#421)

* Remove #[allow(dead_code)] from the codebase

* Remove: dead_code, move: avroschema fn to test

* Fix checks and code style, remove unused code

* Change function name

* ci: use official typos github action (#426)

* feat: support lower_bound&&upper_bound for parquet writer (#383)

* refactor: Implement ArrowAsyncFileWriter directly to remove tokio (#427)

* refactor: Implement ArrowAsyncFileWriter directly to remove tokio

Signed-off-by: Xuanwo <[email protected]>

* Make build pass

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* chore: Don't enable reqwest default features (#432)

* refactor(catalogs/rest): Split user config and runtime config (#431)

* refactor(catalogs/rest): Split user config and runtime config

Signed-off-by: Xuanwo <[email protected]>

* Sort cargo

Signed-off-by: Xuanwo <[email protected]>

* Fix unit tests

Signed-off-by: Xuanwo <[email protected]>

* Remove default feature of tokio

Signed-off-by: Xuanwo <[email protected]>

* return error here

Signed-off-by: Xuanwo <[email protected]>

* Return error if cred doesn't exist

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* feat: runtime module (#233)

* temp runtime

* POC

* fix chrono

* fix dep

* refine module

* refactor to use a deadly simple way

* allow dead_code

* add license

* fix clippy and tests

* clean code

* undo

* add async-std ci test

* rm tokio dev-dep

* make tokio dev dep

* fix sort

* rm tokio dev

* fix: Fix namespace identifier in url (#435)

* fix: Fix namespace identifier in url

* Remove table encoding

* refactor(io): Split io into smaller mods (#438)

* refactor(io): Split io into smaller mods

Signed-off-by: Xuanwo <[email protected]>

* Fix test

Signed-off-by: Xuanwo <[email protected]>

* Format

Signed-off-by: Xuanwo <[email protected]>

* Fix cap

Signed-off-by: Xuanwo <[email protected]>

* Remove not used deps

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* chore: Use once_cell to replace lazy_static (#443)

* chore: Use once_cell to replace lazy_static

Signed-off-by: Xuanwo <[email protected]>

* Format toml

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* fix: Fix build while no-default-features enabled (#442)

* fix: Fix build while no-default-features enabled

Signed-off-by: Xuanwo <[email protected]>

* Fix clippy

Signed-off-by: Xuanwo <[email protected]>

* Add ci for no default features

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* chore(deps): Bump crate-ci/typos from 1.22.9 to 1.23.1 (#447)

Bumps [crate-ci/typos](https://github.com/crate-ci/typos) from 1.22.9 to 1.23.1.
- [Release notes](https://github.com/crate-ci/typos/releases)
- [Changelog](https://github.com/crate-ci/typos/blob/master/CHANGELOG.md)
- [Commits](https://github.com/crate-ci/typos/compare/v1.22.9...v1.23.1)

---
updated-dependencies:
- dependency-name: crate-ci/typos
  dependency-type: direct:production
  update-type: version-update:semver-minor
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* docs: Refactor the README to be more user-oriented (#444)

* docs: Refactor the README to be more user-oriented

Signed-off-by: Xuanwo <[email protected]>

* Apply suggestions from code review

Co-authored-by: Fokko Driesprong <[email protected]>

* Polish

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>
Co-authored-by: Fokko Driesprong <[email protected]>

* feat: Add cargo machete (#448)

* adding cargo machete to check unused dependencies

* remove default values

* adding a tag version instead of main

* running cargo machete natively

* removing unused dependency urlencoding

* bug fixes

* chore: Use nightly toolchain for check (#445)

* chore: Use nightly toolchain for check

* Fix check

* Fix clippy finds

* Make rustfmt happy

* Make rustfmt happy

* Update github actions

* Use action builder since apache doesn't allow external actions

* Fix comments

* Fix README.md

* reuse docker container to save compute resources (#428)

* reuse docker container to save compute resources

* add lazy resuse docker compose

* refactor test fixture: the docker compose init is reused

* use ctor and dtor to start docker compose and destory docker compose

* fix cargo fmt check

* fix cargo clippy

* fix cargo fmt

* fix cargo sort

* add namespace for datafusion test

* add empty check for list glue catalog namespace

---------

Co-authored-by: thexiay <[email protected]>

* feat: Add macos runner for ci (#441)

* feat: Add macos runner for ci

* feat: Add publish for macOS

* reset the publish.yml

* feat: add macOS for check ci

* remove the macOS for unit ci

* chore: remove compose obsolete version (#452) (#454)

reference: https://docs.docker.com/compose/compose-file/04-version-and-name/#version-top-level-element-obsolete

* Refactor file_io_s3_test.rs (#455)

* chore(deps): Bump crate-ci/typos from 1.23.1 to 1.23.2 (#457)

Bumps [crate-ci/typos](https://github.com/crate-ci/typos) from 1.23.1 to 1.23.2.
- [Release notes](https://github.com/crate-ci/typos/releases)
- [Changelog](https://github.com/crate-ci/typos/blob/master/CHANGELOG.md)
- [Commits](https://github.com/crate-ci/typos/compare/v1.23.1...v1.23.2)

---
updated-dependencies:
- dependency-name: crate-ci/typos
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* remove binary serialize in literal (#456)

Co-authored-by: ZENOTME <[email protected]>

* fix: Hms test on macos should use correct arch (#461)

* Fix ManifestFile length calculation (#466)

* chore(deps): Update typed-builder requirement from ^0.18 to ^0.19 (#473)

---
updated-dependencies:
- dependency-name: typed-builder
  dependency-type: direct:production
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* fix: use avro fixed to represent decimal (#472)

fix #144

Signed-off-by: xxchan <[email protected]>

* chore(catalog): Deprecate rest.authorization-url in favor of oauth2-server-uri (#480)

* fix: Transform::Day maps to Date rather than Int for consistency with reference implementation (#479)

Issue: https://github.com/apache/iceberg-rust/issues/478

* feat(iceberg): Add memory file IO support (#481)

* feat(iceberg): Add memory file IO support

Signed-off-by: Xuanwo <[email protected]>

* Fix typo

Signed-off-by: Xuanwo <[email protected]>

* Add comments for memory file io

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* Add in-memory catalog implementation (#475)

* feat: Add in-memory catalog

* Make clippy happy

* Make cargo sort happy

* Fix README links

* Configurable file_io

* Avoid nightly features

* Remove TempFile

* Use futures::lock::Mutex instead

* Minor clean up

* Make root configurable in FS FileIO and remove default_table_root_location from Catalog

* Revert "Make root configurable in FS FileIO and remove default_table_root_location from Catalog"

This reverts commit 807dd4cf649b5c367f25afc59f99341d6995c337.

* Remove default_table_root_location from Catalog and explicitly configure a location for tables in unit tests

* lowercase catalog

* Use default instead of new

* Change references to memory

* chore: Enable new rust code format settings (#483)

* chore: Enable new format settings

Signed-off-by: Xuanwo <[email protected]>

* Format

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* docs: Generate rust API docs (#486)

Signed-off-by: Xuanwo <[email protected]>

* chore: Fix format of recent PRs (#487)

Signed-off-by: Xuanwo <[email protected]>

* Rename folder to memory (#490)

* chore(deps): Bump crate-ci/typos from 1.23.2 to 1.23.5 (#493)

Bumps [crate-ci/typos](https://github.com/crate-ci/typos) from 1.23.2 to 1.23.5.
- [Release notes](https://github.com/crate-ci/typos/releases)
- [Changelog](https://github.com/crate-ci/typos/blob/master/CHANGELOG.md)
- [Commits](https://github.com/crate-ci/typos/compare/v1.23.2...v1.23.5)

---
updated-dependencies:
- dependency-name: crate-ci/typos
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* View Spec implementation (#331)

* Add support for ViewSpec

* Fix typos

* Fix typos

* clippy is always right

* Add tests

* Remove new_view_version test function

* Remove append_version

* View Representations Struct

* ViewRepresentation case insensitive

* Add fallible methods for ViewRepresentationsBuilder

* Add tests for fallibe ViewRepresentationsBuilder methods

* Introduce ViewVersionId as i32

* Iterator for &'a ViewRepresentations

* Improve comments

Co-authored-by: Renjie Liu <[email protected]>

* Add test_view_metadata_v1_file_valid

* Fix view_version iter

* Remove ViewRepresentationsBuilder

* Fix comment

* Timestamp error handling

* Fallible Timestamp Conversion from Millis

* Fix Initial view Version = 1

* Cleanup

* Hide ViewMetadata iter() type

* timestamp_ms_to_utc -> error.rs

* TableMetadata timestamp conversion -> utility function

* Improve error context

* timestamp_ms_to_utc: LocalResult::None -> DataInvalid

* Fix obsolete comment

* ViewRepresentation::SqlViewRepresentation -> ::Sql

* Fix broken clippy from rebase

---------

Co-authored-by: Renjie Liu <[email protected]>

* fix: Return error on reader task (#498)

* chore: Bump OpenDAL to 0.48 (#500)

* chore: Bump OpenDAL to 0.48

Signed-off-by: Xuanwo <[email protected]>

* Format toml

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* add check compatible func for primitive type (#492)

Co-authored-by: ZENOTME <[email protected]>

* refactor(iceberg): Remove an extra config parse logic (#499)

* refactor(iceberg): Remove an extra config parse logic

Signed-off-by: Xuanwo <[email protected]>

* Format toml

Signed-off-by: Xuanwo <[email protected]>

* reduce some allocs

Signed-off-by: Xuanwo <[email protected]>

* Cleanup more

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* feat: permit Datum Date<->Int type conversion (#496)

Required for correct handling of partitions with Transform::Date

* Add additional S3 FileIO Attributes (#505)

* Add additional S3 FileIO Attributes

* Remove custom S3SSEType

* docs: Add links to dev docs (#508)

* docs: Add links to dev docs

Signed-off-by: Xuanwo <[email protected]>

* Add links

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* chore: Remove typo in README (#509)

* chore: Remove typo in README

Signed-off-by: Xuanwo <[email protected]>

* Fix link

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* feat: podman support (#489)

* feat: improve docker/podman get OsArch compatibility

* fix: replace deprecated 'links' keyword in docker compose

* refactor: use IpAddr and SocketAddr when able

* docs: add podman documentation

* address PR documentation comments

* address pr comments on tests

* Address pr comments, properly handle result with match

* address pr comments, refactor get_cmd_output

* move podman instr to docs/contributing

* address pr comment, show detailed err msg

---------

Co-authored-by: Alex Yin <[email protected]>

* feat(table): Add debug and clone trait to static table struct (#510)

* Use namespace location or warehouse location if table location is missing (#511)

* chore(deps): Bump crate-ci/typos from 1.23.5 to 1.23.6 (#521)

* Concurrent table scans (#373)

* feat: concurrent table scans

* refactor: remove TableScanConfig.

* refactor: replace num_cpus with thread::available_parallelism (#526)

* Fix: MappedLocalTime should not be exposed (#529)

* feat: Establish subproject pyiceberg_core (#518)

Signed-off-by: Xuanwo <[email protected]>

* fix: complete miss attribute for map && list in avro schema (#411)

* add miss attr in list/map avro schema

* refine error handle

* fix unused warn

* fix typos

* update avro and unittest

* refine check_schema_conversion

---------

Co-authored-by: ZENOTME <[email protected]>

* arrow/schema.rs: refactor tests (#531)

* arrow/schema.rs: refactor tests

Signed-off-by: Shirly <[email protected]>

* *:address comments

Signed-off-by: Shirly <[email protected]>

---------

Signed-off-by: Shirly <[email protected]>

* feat: initialise SQL Catalog (#524)

* feat: initialise SQL Catalog

Signed-off-by: callum-ryan <[email protected]>

* fix: remove rls-rustls

Signed-off-by: callum-ryan <[email protected]>

* feat: change to SqlBindStyle and rename consts

Signed-off-by: callum-ryan <[email protected]>

---------

Signed-off-by: callum-ryan <[email protected]>

* chore(deps): Bump actions/setup-python from 4 to 5 (#536)

Bumps [actions/setup-python](https://github.com/actions/setup-python) from 4 to 5.
- [Release notes](https://github.com/actions/setup-python/releases)
- [Commits](https://github.com/actions/setup-python/compare/v4...v5)

---
updated-dependencies:
- dependency-name: actions/setup-python
  dependency-type: direct:production
  update-type: version-update:semver-major
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* support session token (#530)

* Simplify PrimitiveLiteral (#502)

* simplify PrimitiveLiteral

* fix test

---------

Co-authored-by: ZENOTME <[email protected]>

* chore: bump opendal to 0.49 (#540)

* feat: support timestamp columns in row filters (#533)

Fixes: https://github.com/apache/iceberg-rust/issues/532

* fix: don't silently drop errors encountered in table scan file planning (#535)

* chore(deps): Update sqlx requirement from 0.7.4 to 0.8.0 (#537)

Updates the requirements on [sqlx](https://github.com/launchbadge/sqlx) to permit the latest version.
- [Changelog](https://github.com/launchbadge/sqlx/blob/main/CHANGELOG.md)
- [Commits](https://github.com/launchbadge/sqlx/compare/v0.7.4...v0.8.0)

---
updated-dependencies:
- dependency-name: sqlx
  dependency-type: direct:production
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Fix main branch building break (#541)

* feat: support for gcs storage (#520)

* chore: include opendal/services-gcs

* feat: basic gcs scaffolding

* feat: populate config parse with basic details

* feat: include docker-compose integration tests

* feat: add extra iceberg properties

* feat: add tests for gcs read/write

These are currently conditional tests with a todo comment using the
test_with proc macro. More work needs to be done on
investigating/potentially expanding OpenDAL to allow unauthenticated
requests to fake-gcs-server. At the moment this always ends up reaching
the final VM metadata check.

* chore: minor cleanup for compose todo

* fix: do not introduce new properties

* feat: infer bucket from path

* chore: add user-project const

* feat: add allow_anonymous for test

* chore: remove test-with dep

* feat: update with allow_anonymous functionality

This requires the opendal allow_anonymous funcitonality with the GCS
service to work.

* ci: use cargo sort

* chore: undo storage-gcs default feature

* feat: include disable_ params for GCS_NO_AUTH

* ci: use storage-all for async-std tests

* revert: use opendal from workspace

Now that v0.49 has been released, this work does not need to pin to a
particular version!

* feat: Allow FileIO to reuse http client (#544)

Signed-off-by: Xuanwo <[email protected]>

* docs: Add an example to scan an iceberg table (#545)

* docs: Add an example to scan an iceberg table

Signed-off-by: Xuanwo <[email protected]>

* Format toml

Signed-off-by: Xuanwo <[email protected]>

---------

Signed-off-by: Xuanwo <[email protected]>

* Concurrent data file fetching and parallel RecordBatch processing (#515)

* feat: concurrent data file fetches, parallel RecordBatch processing

* refactor: centralize infallible `available_parallelism` fn. Use better channel size limit in arrow read

* doc: Add statement for contributors to avoid force push as much as possible (#546)

* chore: Bump datafusion to 41 (#548)

Signed-off-by: Xuanwo <[email protected]>

* feat: Partition Binding and safe PartitionSpecBuilder (#491)

* Initial commit

* Fixes

* Replace UnboundPartitionSpec Builder

* Fix tests, allow year, month day partition

* Comments

* typos

* Fix UnboundBuild setting partition_id

* Add test for unbound spec without partition ids

* Fix into_unbound fn name

* Split bound & unbound Partition builder, change add_partition_fields

* Improve comment

* Fix fmt

* Review fixes

* Remove partition_names() HashSet creation

* Bump to version 0.3.0 (#549)

* Bump to version 0.3.0

Signed-off-by: Xuanwo <[email protected]>

* regen

Signed-off-by: Xuanwo <[email protected]>

* Fix typo

Signed-off-by: Xuanwo <[email protected]>

* Update CHANGELOG.md

---------

Signed-off-by: Xuanwo <[email protected]>

* io: add support for role arn and external id s3 props (#553)

Add support for client.assume-role.arn and
client.assume-role.external-id s3 config properties.

Partial fix for #527

* fix: ensure S3 and GCS integ tests are conditionally compiled only when the storage-s3 and storage-gcs features are enabled (#552)

* docs: fix main iceberg example (#554)

* io: add support to set assume role session name (#555)

Partial fix for #527

* test: refactor datafusion test with memory catalog (#557)

* add memory catalog

* fix style

* fix style

* add clean job in Makefile (#561)

* docs: Fix build website permission changed (#564)

* Object Cache: caches parsed Manifests and ManifestLists for performance (#512)

* feat: adds ObjectCache, to cache Manifests and ManifestLists

* refactor: change obj cache method names and use more readable default usize value

* chore: improve error message

Co-authored-by: Renjie Liu <[email protected]>

* fix: change object cache retrieval method visibility

Co-authored-by: Renjie Liu <[email protected]>

* feat: improved error message in object cache get_manifest

* test(object-cache): add unit tests for object cache manifest and manifest list retrieval

* fix: ensure that object cache insertions are weighted by size

* test: fix test typo

* fix: ensure object cache weight is that of the wrapped item, not the Arc

---------

Co-authored-by: Renjie Liu <[email protected]>

* Update the paths (#569)

This is in line with the paths above, and also with the previous version:

https://dist.apache.org/repos/dist/release/iceberg/

* docs: Add links for released crates (#570)

Signed-off-by: Xuanwo <[email protected]>

* Python: Use hatch for dependency management (#572)

* Ensure that RestCatalog passes user config to FileIO (#476)

* fix: ensure that RestCatalog passes user config to FileIO

* docs: added some doc comments to clarify override order for config

* Move `zlib` and `unicode` licenses to `allow` (#566)

Both licenses can be moved to the `allowed` section:

- **adler32** [ships](https://github.com/remram44/adler32-rs/blob/master/LICENSE) with a **zlib** license and is a category A-license
- **unicode-ident** ships with a **UNICODE, INC. LICENSE AGREEMENT - DATA FILES AND SOFTWARE** which is also a category A-license

The **ring** license is a bit [more involved](https://github.com/briansmith/ring/blob/main/LICENSE) and carries a lot of history, I think it is best to keep that as an exception for now, since the OpenSSL license is also not explicitly listed on the ASF page. I don't see anything alarming in the `LICENSE` file.

ASF page on the subject: https://www.apache.org/legal/resolved.html#category-a

* website: Update links for 0.3.0 (#573)

Signed-off-by: Xuanwo <[email protected]>

* feat(timestamp_ns): Implement timestamps with nanosecond precision (#542)

* feat(timestamp_ns): first commit

* feat(timestamp_ns): Add mappings for timestamp_ns/timestamptz_ns

* feat(timestamp_ns): Remove unused dep

* feat(timestamp_ns): Fix unit test

* feat(timestamp_ns): Fix test_all_type_for_write()

* feat(timestamp_ns): fix test_transform_days_literal

* feat(timestamp_ns): fix math for timestamptz_nanos

* chore: formatting

* chore: formatting

* chore: Appease clippy

---------

Co-authored-by: Timothy Maloney <[email protected]>

* fix: correct partition-id to field-id in UnboundPartitionField (#576)

* correct partition-id to field id in PartitionSpec

* correct partition-id to field id in PartitionSpec

* correct partition-id to field id in PartitionSpec

* xx

* fix: Update sqlx from 0.8.0 to 0.8.1 (#584)

* chore(deps): Update typed-builder requirement from 0.19 to 0.20 (#582)

---
updated-dependencies:
- dependency-name: typed-builder
  dependency-type: direct:production
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* Expose Transforms to Python Binding (#556)

* bucket transform rust binding

* format

* poetry x maturin

* ignore poetry.lock in license check

* update bindings_python_ci to use makefile

* newline

* https://github.com/python-poetry/poetry/pull/9135

* use hatch instead of poetry

* refactor

* revert licenserc change

* adopt review feedback

* comments

* unused dependency

* adopt review comment

* newline

* I like this approach a lot better

* more tests

* chore(deps): Bump crate-ci/typos from 1.23.6 to 1.24.1 (#583)

* Table Scan: Add Row Group Skipping (#558)

* feat(scan): add row group and page index row selection filtering

* fix(row selection): off-by-one error

* feat: remove row selection to defer to a second PR

* feat: better min/max val conversion in RowGroupMetricsEvaluator

* test(row_group_filtering): first three tests

* test(row_group_filtering): next few tests

* test: add more tests for RowGroupMetricsEvaluator

* chore: refactor test assertions to silence clippy lints

* refactor: consolidate parquet stat min/max parsing in one place

* chore: bump crate-ci/typos to 1.24.3 (#598)

* feat: SQL Catalog - namespaces (#534)

* feat: SQL Catalog - namespaces

Signed-off-by: callum-ryan <[email protected]>

* feat: use transaction for updates and creates

Signed-off-by: callum-ryan <[email protected]>

* fix: pull out query param builder to fn

Signed-off-by: callum-ryan <[email protected]>

* feat: add drop and tests

Signed-off-by: callum-ryan <[email protected]>

* fix: String to str, remove pub and optimise query builder

Signed-off-by: callum-ryan <[email protected]>

* fix: nested match, remove ok()

Signed-off-by: callum-ryan <[email protected]>

* fix: remove pub, add set, add comments

Signed-off-by: callum-ryan <[email protected]>

* fix: refactor list_namespaces slightly

Signed-off-by: callum-ryan <[email protected]>

* fix: add default properties to all new namespaces

Signed-off-by: callum-ryan <[email protected]>

* fix: remove check for nested namespace

Signed-off-by: callum-ryan <[email protected]>

* chore: add more comments to the CatalogConfig to explain bind styles

Signed-off-by: callum-ryan <[email protected]>

* fix: edit test for nested namespaces

Signed-off-by: callum-ryan <[email protected]>

---------

Signed-off-by: callum-ryan <[email protected]>

* feat: Add more fields in FileScanTask (#609)

Signed-off-by: Xuanwo <[email protected]>

* chore(deps): Bump crate-ci/typos from 1.24.3 to 1.24.5 (#616)

Bumps [crate-ci/typos](https://github.com/crate-ci/typos) from 1.24.3 to 1.24.5.
- [Release notes](https://github.com/crate-ci/typos/releases)
- [Changelog](https://github.com/crate-ci/typos/blob/master/CHANGELOG.md)
- [Commits](https://github.com/crate-ci/typos/compare/v1.24.3...v1.24.5)

---
updated-dependencies:
- dependency-name: crate-ci/typos
  dependency-type: direct:production
  update-type: version-update:semver-patch
...

Signed-off-by: dependabot[bot] <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>

* fix: Less Panics for Snapshot timestamps (#614)

* feat: partition compatibility (#612)

* Partition compatability

* Partition compatability

* Rename compatible_with -> is_compatible_with

* feat: SortOrder methods should take schema ref if possible (#613)

* SortOrder methods should take schema ref if possible

* Fix test type

* with_order_id should not take reference

* feat: add `client.region` (#623)

* fix: Correctly calculate highest_field_id in schema (#590)

---------

Signed-off-by: Xuanwo <[email protected]>
Signed-off-by: dependabot[bot] <[email protected]>
Signed-off-by: Ruihang Xia <[email protected]>
Signed-off-by: TennyZhuang <[email protected]>
Signed-off-by: xxchan <[email protected]>
Signed-off-by: Shirly <[email protected]>
Signed-off-by: callum-ryan <[email protected]>
Co-authored-by: Xuanwo <[email protected]>
Co-authored-by: Renjie Liu <[email protected]>
Co-authored-by: Fokko Driesprong <[email protected]>
Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com>
Co-authored-by: hiirrxnn <[email protected]>
Co-authored-by: Tyler Schauer <[email protected]>
Co-authored-by: ZENOTME <[email protected]>
Co-authored-by: ZENOTME <[email protected]>
Co-authored-by: Chengxu Bian <[email protected]>
Co-authored-by: JanKaul <[email protected]>
Co-authored-by: Manu Zhang <[email protected]>
Co-authored-by: Yue Deng <[email protected]>
Co-authored-by: Dinesh Phuyel <[email protected]>
Co-authored-by: ZHENGLIN LI <[email protected]>
Co-authored-by: Mark Grey <[email protected]>
Co-authored-by: Farooq Qaiser <[email protected]>
Co-authored-by: Scott Donnelly <[email protected]>
Co-authored-by: Shabana Baig <[email protected]>
Co-authored-by: stream2000 <[email protected]>
Co-authored-by: fuqijun <[email protected]>
Co-authored-by: Liang-Chi Hsieh <[email protected]>
Co-authored-by: Marvin Lanhenke <[email protected]>
Co-authored-by: mlanhenke <[email protected]>
Co-authored-by: 张林伟 <[email protected]>
Co-authored-by: Alon Agmon <[email protected]>
Co-authored-by: TennyZhuang <[email protected]>
Co-authored-by: Ruihang Xia <[email protected]>
Co-authored-by: Himadri Pal <[email protected]>
Co-authored-by: hpal <[email protected]>
Co-authored-by: Howie Wang <[email protected]>
Co-authored-by: QuakeWang <[email protected]>
Co-authored-by: Ajay Gupte <[email protected]>
Co-authored-by: Christian <[email protected]>
Co-authored-by: Matiukhin Vlad <[email protected]>
Co-authored-by: Jack <[email protected]>
Co-authored-by: Christian <[email protected]>
Co-authored-by: Vivek Khatri <[email protected]>
Co-authored-by: ZhangJian He <[email protected]>
Co-authored-by: Vipul Vaibhaw <[email protected]>
Co-authored-by: thexia <[email protected]>
Co-authored-by: thexiay <[email protected]>
Co-authored-by: yinheli <[email protected]>
Co-authored-by: tom <[email protected]>
Co-authored-by: xxchan <[email protected]>
Co-authored-by: Andre Luis Anastacio <[email protected]>
Co-authored-by: Andre Luis Anastacio <[email protected]>
Co-authored-by: Alex Yin <[email protected]>
Co-authored-by: Alex Yin <[email protected]>
Co-authored-by: SteveLauC <[email protected]>
Co-authored-by: Shirly <[email protected]>
Co-authored-by: Callum Ryan <[email protected]>
Co-authored-by: Tobias Pütz <[email protected]>
Co-authored-by: Matheus Alcantara <[email protected]>
Co-authored-by: Matheus Alcantara <[email protected]>
Co-authored-by: FANNG <[email protected]>
Co-authored-by: ChinoUkaegbu <[email protected]>
Co-authored-by: Sung Yun <[email protected]>
Co-authored-by: Timothy Maloney <[email protected]>
Co-authored-by: Timothy Maloney <[email protected]>
Co-authored-by: Søren Dalby Larsen <[email protected]>
shaeqahmed pushed a commit to matanolabs/iceberg-rust that referenced this pull request Dec 9, 2024
* Add support for ViewSpec

* Fix typos

* Fix typos

* clippy is always right

* Add tests

* Remove new_view_version test function

* Remove append_version

* View Representations Struct

* ViewRepresentation case insensitive

* Add fallible methods for ViewRepresentationsBuilder

* Add tests for fallibe ViewRepresentationsBuilder methods

* Introduce ViewVersionId as i32

* Iterator for &'a ViewRepresentations

* Improve comments

Co-authored-by: Renjie Liu <[email protected]>

* Add test_view_metadata_v1_file_valid

* Fix view_version iter

* Remove ViewRepresentationsBuilder

* Fix comment

* Timestamp error handling

* Fallible Timestamp Conversion from Millis

* Fix Initial view Version = 1

* Cleanup

* Hide ViewMetadata iter() type

* timestamp_ms_to_utc -> error.rs

* TableMetadata timestamp conversion -> utility function

* Improve error context

* timestamp_ms_to_utc: LocalResult::None -> DataInvalid

* Fix obsolete comment

* ViewRepresentation::SqlViewRepresentation -> ::Sql

* Fix broken clippy from rebase

---------

Co-authored-by: Renjie Liu <[email protected]>
@c-thiel c-thiel deleted the ft/views branch January 10, 2025 06:56
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants