Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-24 Thread via GitHub


Fokko commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1577458027


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,466 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use crate::{Error, ErrorKind};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+// The partition_schema's schema_id is set to the partition
+// spec's spec_id here, and used to perform a sanity check
+// during eval to confirm that it matches the spec_id

Review Comment:
   Yes, you're right. Thanks for pointing that out  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-24 Thread via GitHub


sdd commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1577440808


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,459 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use crate::{Error, ErrorKind};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_schema_id(partition_spec.spec_id)
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());
+let unbound_partition_filter = inclusive_projection.project()?;
+
+let partition_filter = unbound_partition_filter
+.rewrite_not()
+.bind(partition_schema_ref.clone(), case_sensitive)?;
+
+Ok(Self {
+partition_schema: partition_schema_ref,
+partition_filter,
+case_sensitive,
+})
+}
+
+/// Evaluate this `ManifestEvaluator`'s filter predicate against the
+/// provided [`ManifestFile`]'s partitions. Used by [`TableScan`] to
+/// see if this `ManifestFile` could possibly contain data that matches
+/// the scan's filter.
+pub(crate) fn eval(, manifest_file: ) -> 
crate::Result {
+if manifest_file.partitions.is_empty() {
+return Ok(true);
+}
+
+if self.partition_schema.schema_id() != 
manifest_file.partition_spec_id {

Review Comment:
   Hi @Fokko. That happens here:
   
   
https://github.com/apache/iceberg-rust/pull/323/files#diff-bbfbe5e334be6c501ba2ca0ddd84d658ff0f3f84f2b5b532212f4e096a09e09bR205-R207



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-24 Thread via GitHub


sdd commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1577440808


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,459 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use crate::{Error, ErrorKind};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_schema_id(partition_spec.spec_id)
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());
+let unbound_partition_filter = inclusive_projection.project()?;
+
+let partition_filter = unbound_partition_filter
+.rewrite_not()
+.bind(partition_schema_ref.clone(), case_sensitive)?;
+
+Ok(Self {
+partition_schema: partition_schema_ref,
+partition_filter,
+case_sensitive,
+})
+}
+
+/// Evaluate this `ManifestEvaluator`'s filter predicate against the
+/// provided [`ManifestFile`]'s partitions. Used by [`TableScan`] to
+/// see if this `ManifestFile` could possibly contain data that matches
+/// the scan's filter.
+pub(crate) fn eval(, manifest_file: ) -> 
crate::Result {
+if manifest_file.partitions.is_empty() {
+return Ok(true);
+}
+
+if self.partition_schema.schema_id() != 
manifest_file.partition_spec_id {

Review Comment:
   
https://github.com/apache/iceberg-rust/pull/323/files#diff-bbfbe5e334be6c501ba2ca0ddd84d658ff0f3f84f2b5b532212f4e096a09e09bR205-R207



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-23 Thread via GitHub


liurenjie1024 commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1577106941


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,466 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use crate::{Error, ErrorKind};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+// The partition_schema's schema_id is set to the partition
+// spec's spec_id here, and used to perform a sanity check
+// during eval to confirm that it matches the spec_id

Review Comment:
   I think the cache is done here: 
https://github.com/apache/iceberg-rust/pull/323



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-23 Thread via GitHub


liurenjie1024 commented on PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#issuecomment-2073844403

   > There are intentionally a lot of `todo!`s in here. The aim is to get this 
PR merged so that more people can contribute implementations for the different 
visitor methods that need implementing, in subsequent PRs.
   
   cc @Fokko I think @sdd is planning to add implementations in following 
smaller prs.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-23 Thread via GitHub


Fokko commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1576805382


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,466 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use crate::{Error, ErrorKind};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+// The partition_schema's schema_id is set to the partition
+// spec's spec_id here, and used to perform a sanity check
+// during eval to confirm that it matches the spec_id
+// of the ManifestFile we're evaluating
+let partition_schema = Schema::builder()
+.with_schema_id(partition_spec.spec_id)
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());
+let unbound_partition_filter = inclusive_projection.project()?;
+
+let partition_filter = unbound_partition_filter
+.rewrite_not()
+.bind(partition_schema_ref.clone(), case_sensitive)?;
+
+Ok(Self {
+partition_schema: partition_schema_ref,
+partition_filter,
+case_sensitive,
+})
+}
+
+/// Evaluate this `ManifestEvaluator`'s filter predicate against the
+/// provided [`ManifestFile`]'s partitions. Used by [`TableScan`] to
+/// see if this `ManifestFile` could possibly contain data that matches
+/// the scan's filter.
+pub(crate) fn eval(, manifest_file: ) -> 
crate::Result {
+if manifest_file.partitions.is_empty() {
+return Ok(true);
+}
+
+// The schema_id of self.partition_schema is set to the
+// spec_id of the partition spec that this ManifestEvaluator
+// was created from in ManifestEvaluator::new
+if self.partition_schema.schema_id() != 
manifest_file.partition_spec_id {
+return Err(Error::new(
+ErrorKind::Unexpected,
+format!(
+"Partition ID for manifest file '{}' does not match 
partition ID for the Scan",
+_file.manifest_path
+),
+));
+}
+
+let mut evaluator = ManifestFilterVisitor::new(self, 
_file.partitions);
+
+visit( evaluator, _filter)
+}
+}
+
+struct ManifestFilterVisitor<'a> {
+manifest_evaluator: &'a ManifestEvaluator,
+partitions: &'a Vec,
+}
+
+impl<'a> ManifestFilterVisitor<'a> {
+fn new(manifest_evaluator: &'a ManifestEvaluator, partitions: &'a 
Vec) -> Self {
+ManifestFilterVisitor {
+manifest_evaluator,
+partitions,
+}
+}
+}
+
+// Remove this annotation once all todos have been removed
+#[allow(unused_variables)]
+impl BoundPredicateVisitor for ManifestFilterVisitor<'_> {
+type T = bool;
+
+fn always_true( self) -> crate::Result {
+Ok(true)
+ 

Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-23 Thread via GitHub


Fokko commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1576800710


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,459 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use crate::{Error, ErrorKind};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_schema_id(partition_spec.spec_id)
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());
+let unbound_partition_filter = inclusive_projection.project()?;
+
+let partition_filter = unbound_partition_filter
+.rewrite_not()
+.bind(partition_schema_ref.clone(), case_sensitive)?;
+
+Ok(Self {
+partition_schema: partition_schema_ref,
+partition_filter,
+case_sensitive,
+})
+}
+
+/// Evaluate this `ManifestEvaluator`'s filter predicate against the
+/// provided [`ManifestFile`]'s partitions. Used by [`TableScan`] to
+/// see if this `ManifestFile` could possibly contain data that matches
+/// the scan's filter.
+pub(crate) fn eval(, manifest_file: ) -> 
crate::Result {
+if manifest_file.partitions.is_empty() {
+return Ok(true);
+}
+
+if self.partition_schema.schema_id() != 
manifest_file.partition_spec_id {

Review Comment:
   As mentioned above, likely, this doesn't match. We need to create an 
evaluator for each `partition_spec_id` that we see.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-23 Thread via GitHub


Fokko commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1576798073


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,466 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use crate::{Error, ErrorKind};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+// The partition_schema's schema_id is set to the partition
+// spec's spec_id here, and used to perform a sanity check
+// during eval to confirm that it matches the spec_id

Review Comment:
   A table can have manifests with different specs. If you write data, it will 
adhere to the current spec. When you evolve the spec, the existing data will 
not be rewritten since Iceberg is lazy by design (you don't want to rewrite a 
petabyte-sized table at once).
   
   In PyIceberg we cache on the spec-id, so we don't re-create all the 
evaluators.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-22 Thread via GitHub


liurenjie1024 merged PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-22 Thread via GitHub


sdd commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1574198729


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,459 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use crate::{Error, ErrorKind};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_schema_id(partition_spec.spec_id)
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());

Review Comment:
   No problem!  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-22 Thread via GitHub


ZENOTME commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1574193520


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,459 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use crate::{Error, ErrorKind};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_schema_id(partition_spec.spec_id)
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());

Review Comment:
   Thanks! I got it!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-22 Thread via GitHub


sdd commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1574191566


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,459 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use crate::{Error, ErrorKind};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_schema_id(partition_spec.spec_id)
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());

Review Comment:
   an _Inclusive projection_ is so named because it matches any file that 
**may** contain **some** rows that match the partition. This is in contrast to 
a _strict projection_, which matches files in which **all** rows within match 
the predicate.
   
   The naming is carried over from the 
[Java](https://iceberg.apache.org/javadoc/1.5.0/org/apache/iceberg/expressions/Projections.ProjectionEvaluator.html)
 and 
[Python](https://github.com/apache/iceberg-python/blob/05d604a2cc21a8b78fc2fe87047a022d97dd2ece/pyiceberg/expressions/visitors.py#L831)
 implementations.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-22 Thread via GitHub


sdd commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1574182261


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,459 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use crate::{Error, ErrorKind};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_schema_id(partition_spec.spec_id)
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());
+let unbound_partition_filter = inclusive_projection.project()?;
+
+let partition_filter = unbound_partition_filter
+.rewrite_not()
+.bind(partition_schema_ref.clone(), case_sensitive)?;
+
+Ok(Self {
+partition_schema: partition_schema_ref,
+partition_filter,
+case_sensitive,
+})
+}
+
+/// Evaluate this `ManifestEvaluator`'s filter predicate against the
+/// provided [`ManifestFile`]'s partitions. Used by [`TableScan`] to
+/// see if this `ManifestFile` could possibly contain data that matches
+/// the scan's filter.
+pub(crate) fn eval(, manifest_file: ) -> 
crate::Result {
+if manifest_file.partitions.is_empty() {
+return Ok(true);
+}
+
+if self.partition_schema.schema_id() != 
manifest_file.partition_spec_id {

Review Comment:
   Done.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-21 Thread via GitHub


ZENOTME commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1574157028


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,459 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use crate::{Error, ErrorKind};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_schema_id(partition_spec.spec_id)
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());

Review Comment:
   I'm curious why calling this `InclusiveProjection` rather than something can 
be expressed `PartitionProjection`.樂



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-21 Thread via GitHub


marvinlanhenke commented on PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#issuecomment-2068159189

   @sdd Thanks a lot. LGTM


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-21 Thread via GitHub


liurenjie1024 commented on PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#issuecomment-2068041946

   cc @Xuanwo @Fokko @marvinlanhenke PTAL


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-21 Thread via GitHub


liurenjie1024 commented on PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#issuecomment-2068041808

   Let's wait to see if others have comments.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-21 Thread via GitHub


liurenjie1024 commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1573750563


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,459 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use crate::{Error, ErrorKind};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_schema_id(partition_spec.spec_id)
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());
+let unbound_partition_filter = inclusive_projection.project()?;
+
+let partition_filter = unbound_partition_filter
+.rewrite_not()
+.bind(partition_schema_ref.clone(), case_sensitive)?;
+
+Ok(Self {
+partition_schema: partition_schema_ref,
+partition_filter,
+case_sensitive,
+})
+}
+
+/// Evaluate this `ManifestEvaluator`'s filter predicate against the
+/// provided [`ManifestFile`]'s partitions. Used by [`TableScan`] to
+/// see if this `ManifestFile` could possibly contain data that matches
+/// the scan's filter.
+pub(crate) fn eval(, manifest_file: ) -> 
crate::Result {
+if manifest_file.partitions.is_empty() {
+return Ok(true);
+}
+
+if self.partition_schema.schema_id() != 
manifest_file.partition_spec_id {

Review Comment:
   I would suggest to add a comment to explain why comparing schema_id against 
partition_spec_id, or store original spec id directly, but it's not a blocker.



##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,393 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+

Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-19 Thread via GitHub


sdd commented on PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#issuecomment-2067105023

   Thanks for the reviews, @marvinlanhenke and @liurenjie1024! All comments 
addressed and ready for re-review  


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-19 Thread via GitHub


sdd commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1572759976


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,393 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+partition_filter: BoundPredicate,

Review Comment:
   Changed to just `filter`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-19 Thread via GitHub


sdd commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1572759092


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,393 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());
+let unbound_partition_filter = 
inclusive_projection.project(_filter)?;
+
+let partition_filter =
+unbound_partition_filter.bind(partition_schema_ref.clone(), 
case_sensitive)?;
+
+Ok(Self {
+partition_schema: partition_schema_ref,
+partition_filter,
+case_sensitive,
+})
+}
+
+/// Evaluate this `ManifestEvaluator`'s filter predicate against the
+/// provided [`ManifestFile`]'s partitions. Used by [`TableScan`] to
+/// see if this `ManifestFile` could possibly contain data that matches
+/// the scan's filter.
+pub(crate) fn eval(, manifest_file: ) -> 
crate::Result {
+if manifest_file.partitions.is_empty() {
+return Ok(true);
+}
+
+struct ManifestFilterVisitor<'a> {

Review Comment:
   Done :-)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-19 Thread via GitHub


sdd commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1572759566


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,393 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());
+let unbound_partition_filter = 
inclusive_projection.project(_filter)?;
+
+let partition_filter =
+unbound_partition_filter.bind(partition_schema_ref.clone(), 
case_sensitive)?;
+
+Ok(Self {
+partition_schema: partition_schema_ref,
+partition_filter,
+case_sensitive,
+})
+}
+
+/// Evaluate this `ManifestEvaluator`'s filter predicate against the
+/// provided [`ManifestFile`]'s partitions. Used by [`TableScan`] to
+/// see if this `ManifestFile` could possibly contain data that matches
+/// the scan's filter.
+pub(crate) fn eval(, manifest_file: ) -> 
crate::Result {

Review Comment:
   Done, with associated test. Is this what you had in mind?



##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,393 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+

Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-19 Thread via GitHub


sdd commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1572758803


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,393 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());
+let unbound_partition_filter = 
inclusive_projection.project(_filter)?;
+
+let partition_filter =
+unbound_partition_filter.bind(partition_schema_ref.clone(), 
case_sensitive)?;

Review Comment:
   Ok, I suppose it's not exactly a huge overhead :-) 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-19 Thread via GitHub


liurenjie1024 commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1572472267


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,393 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());
+let unbound_partition_filter = 
inclusive_projection.project(_filter)?;
+
+let partition_filter =
+unbound_partition_filter.bind(partition_schema_ref.clone(), 
case_sensitive)?;
+
+Ok(Self {
+partition_schema: partition_schema_ref,
+partition_filter,
+case_sensitive,
+})
+}
+
+/// Evaluate this `ManifestEvaluator`'s filter predicate against the
+/// provided [`ManifestFile`]'s partitions. Used by [`TableScan`] to
+/// see if this `ManifestFile` could possibly contain data that matches
+/// the scan's filter.
+pub(crate) fn eval(, manifest_file: ) -> 
crate::Result {
+if manifest_file.partitions.is_empty() {
+return Ok(true);
+}
+
+struct ManifestFilterVisitor<'a> {
+manifest_evaluator: &'a ManifestEvaluator,
+partitions: &'a Vec,
+}
+
+impl<'a> ManifestFilterVisitor<'a> {
+fn new(
+manifest_evaluator: &'a ManifestEvaluator,
+partitions: &'a Vec,
+) -> Self {
+ManifestFilterVisitor {
+manifest_evaluator,
+partitions,
+}
+}
+}
+
+// Remove this annotation once all todos have been removed
+#[allow(unused_variables)]
+impl BoundPredicateVisitor for ManifestFilterVisitor<'_> {
+type T = bool;
+
+fn always_true( self) -> crate::Result {

Review Comment:
   As our previous discussion, we should not concret type rather `Self::T` to 
make it more readable.



##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,393 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR 

Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-19 Thread via GitHub


sdd commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1572265647


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,393 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());
+let unbound_partition_filter = 
inclusive_projection.project(_filter)?;
+
+let partition_filter =
+unbound_partition_filter.bind(partition_schema_ref.clone(), 
case_sensitive)?;

Review Comment:
   Similarly with InclusiveProjection, since we apply rewrite-not when a 
Predicate is passed to the TableScan as it gets built, we've already guaranteed 
that there are no Not nodes by the time we get here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-19 Thread via GitHub


sdd commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1572263328


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,393 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());
+let unbound_partition_filter = 
inclusive_projection.project(_filter)?;
+
+let partition_filter =
+unbound_partition_filter.bind(partition_schema_ref.clone(), 
case_sensitive)?;
+
+Ok(Self {
+partition_schema: partition_schema_ref,
+partition_filter,
+case_sensitive,
+})
+}
+
+/// Evaluate this `ManifestEvaluator`'s filter predicate against the
+/// provided [`ManifestFile`]'s partitions. Used by [`TableScan`] to
+/// see if this `ManifestFile` could possibly contain data that matches
+/// the scan's filter.
+pub(crate) fn eval(, manifest_file: ) -> 
crate::Result {
+if manifest_file.partitions.is_empty() {
+return Ok(true);
+}
+
+struct ManifestFilterVisitor<'a> {

Review Comment:
   I was following the pattern of index_parents in schema.rs when we do this 
already. I don't really mind either way though 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-19 Thread via GitHub


marvinlanhenke commented on code in PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#discussion_r1572246030


##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,393 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their partition summary matches a 
provided
+/// [`BoundPredicate`]. Used by [`TableScan`] to filter down the list of 
[`ManifestFile`]s
+/// in which data might be found that matches the TableScan's filter.
+pub(crate) struct ManifestEvaluator {
+partition_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+}
+
+impl ManifestEvaluator {
+pub(crate) fn new(
+partition_spec: PartitionSpecRef,
+table_schema: SchemaRef,
+partition_filter: BoundPredicate,
+case_sensitive: bool,
+) -> crate::Result {
+let partition_type = partition_spec.partition_type(_schema)?;
+
+// this is needed as SchemaBuilder.with_fields expects an iterator over
+// Arc rather than 
+let cloned_partition_fields: Vec<_> =
+partition_type.fields().iter().map(Arc::clone).collect();
+
+let partition_schema = Schema::builder()
+.with_fields(cloned_partition_fields)
+.build()?;
+
+let partition_schema_ref = Arc::new(partition_schema);
+
+let mut inclusive_projection = 
InclusiveProjection::new(partition_spec.clone());
+let unbound_partition_filter = 
inclusive_projection.project(_filter)?;
+
+let partition_filter =
+unbound_partition_filter.bind(partition_schema_ref.clone(), 
case_sensitive)?;
+
+Ok(Self {
+partition_schema: partition_schema_ref,
+partition_filter,
+case_sensitive,
+})
+}
+
+/// Evaluate this `ManifestEvaluator`'s filter predicate against the
+/// provided [`ManifestFile`]'s partitions. Used by [`TableScan`] to
+/// see if this `ManifestFile` could possibly contain data that matches
+/// the scan's filter.
+pub(crate) fn eval(, manifest_file: ) -> 
crate::Result {
+if manifest_file.partitions.is_empty() {
+return Ok(true);
+}
+
+struct ManifestFilterVisitor<'a> {

Review Comment:
   nit: although ManifestFilterVisitor is probably only used in `eval`, do we 
really want to nest it inside the fn? 
   Personally I find it somewhat harder to read - but this is just my personal 
view. Perhaps others can comment as well?



##
crates/iceberg/src/expr/visitors/manifest_evaluator.rs:
##
@@ -0,0 +1,393 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+use crate::expr::visitors::bound_predicate_visitor::{visit, 
BoundPredicateVisitor};
+use crate::expr::visitors::inclusive_projection::InclusiveProjection;
+use crate::expr::{Bind, BoundPredicate, BoundReference};
+use crate::spec::{Datum, FieldSummary, ManifestFile, PartitionSpecRef, Schema, 
SchemaRef};
+use fnv::FnvHashSet;
+use std::sync::Arc;
+
+/// Evaluates [`ManifestFile`]s to see if their 

Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-19 Thread via GitHub


sdd commented on PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#issuecomment-2065971864

   There are intentionally a lot of `todo!`s in here. The aim is to get this PR 
merged so that more people can contribute implementations for the different 
visitor methods that need implementing, in subsequent PRs.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



Re: [PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-19 Thread via GitHub


sdd commented on PR #322:
URL: https://github.com/apache/iceberg-rust/pull/322#issuecomment-2065967273

   @Fokko @liurenjie1024 @marvinlanhenke: I've rebased this on top of main now 
that the `InclusiveProjection` has been merged and it would be good to get some 
initial feedback.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org



[PR] Add `ManifestEvaluator`, used to filter manifests in table scans [iceberg-rust]

2024-04-04 Thread via GitHub


sdd opened a new pull request, #322:
URL: https://github.com/apache/iceberg-rust/pull/322

   This PR has been broken out of 
https://github.com/apache/iceberg-rust/pull/241 as that PR was getting too 
large.
   
   It depends on https://github.com/apache/iceberg-rust/pull/320 and 
https://github.com/apache/iceberg-rust/pull/321.
   
   It introduces `ManfestEvaluator`, which is used to apply the filter 
predicate from a table scan to a `ManifestFile` to see if it should be filtered 
out of the scan.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


-
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org