[GitHub] [incubator-iceberg] aokolnychyi commented on a change in pull request #351: Extend Iceberg with a way to overwrite files for eager updates/deletes
aokolnychyi commented on a change in pull request #351: Extend Iceberg with a way to overwrite files for eager updates/deletes URL: https://github.com/apache/incubator-iceberg/pull/351#discussion_r314233769 ## File path: core/src/main/java/org/apache/iceberg/OverwriteData.java ## @@ -88,6 +121,40 @@ public OverwriteFiles validateAddedFiles() { } } +if (conflictingAppendsRowFilter != null) { + PartitionSpec spec = writeSpec(); + Expression inclusiveExpr = Projections.inclusive(spec).project(conflictingAppendsRowFilter); + Evaluator inclusive = new Evaluator(spec.partitionType(), inclusiveExpr); + + List newFiles = collectNewFiles(base); + for (DataFile newFile : newFiles) { +// we do partition-level conflict resolution right now +// we can enhance it by leveraging column stats and MetricsEvaluator +ValidationException.check( +!inclusive.eval(newFile.partition()), +"A conflicting file was appended that matches filter '%s': %s", +conflictingAppendsRowFilter, newFile.path()); + } +} + return super.apply(base); } + + private List collectNewFiles(TableMetadata meta) { +List newFiles = new ArrayList<>(); + +Long currentSnapshotId = meta.currentSnapshot() == null ? null : meta.currentSnapshot().snapshotId(); +while (currentSnapshotId != null && !currentSnapshotId.equals(readSnapshotId)) { + Snapshot currentSnapshot = meta.snapshot(currentSnapshotId); + + if (currentSnapshot == null) { +throw new ValidationException("Cannot find snapshot %d. Was it expired?", currentSnapshotId); Review comment: I think `meta.currentSnapshot` can be `null` if the overwrite happens on an empty table. In that case, the read snapshot is also `null`, so it should work correctly. We have a test for this as well. Such a situation might happen if we use this API for eager `MERGE INTO` operations. Will update the error message. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] aokolnychyi commented on a change in pull request #351: Extend Iceberg with a way to overwrite files for eager updates/deletes
aokolnychyi commented on a change in pull request #351: Extend Iceberg with a way to overwrite files for eager updates/deletes URL: https://github.com/apache/incubator-iceberg/pull/351#discussion_r314234844 ## File path: api/src/main/java/org/apache/iceberg/OverwriteFiles.java ## @@ -74,5 +90,22 @@ * * @return this for method chaining */ - OverwriteFiles validateAddedFiles(); + OverwriteFiles validateAddedFilesMatchRowFilter(); + + /** + * Enables validation of files that are added concurrently. + * + * This method should be called when the table is queried to determine files to delete/append. + * If a concurrent operation commits a new file after the data was read and that file might + * contain rows matching the specified row filter, the overwrite operation will detect + * this during retries and fail. + * + * Calling this method with a correct row filter is required to maintain serializable isolation. + * Otherwise, the isolation level will be snapshot isolation. + * + * @param readSnapshotId the snapshot id that was used to read the data + * @param rowFilter an expression on rows in the table + * @return this for method chaining + */ + OverwriteFiles validateNoConflictingAppends(Long readSnapshotId, Expression rowFilter); Review comment: Allowing `null` was intentional to handle cases when the table is empty. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] aokolnychyi commented on a change in pull request #351: Extend Iceberg with a way to overwrite files for eager updates/deletes
aokolnychyi commented on a change in pull request #351: Extend Iceberg with a way to overwrite files for eager updates/deletes URL: https://github.com/apache/incubator-iceberg/pull/351#discussion_r314236135 ## File path: api/src/main/java/org/apache/iceberg/OverwriteFiles.java ## @@ -24,18 +24,17 @@ import org.apache.iceberg.expressions.Projections; /** - * API for overwriting files in a table by filter expression. + * API for overwriting files in a table. * * This API accumulates file additions and produces a new {@link Snapshot} of the table by replacing - * all the files that match the filter expression with the set of additions. This operation is used - * to implement idempotent writes that always replace a section of a table with new data. + * all the deleted files with the set of additions. This operation is used to implement idempotent + * writes that always replace a section of a table with new data or update/delete operations that + * eagerly overwrite files. Review comment: Added this below. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] aokolnychyi commented on a change in pull request #351: Extend Iceberg with a way to overwrite files for eager updates/deletes
aokolnychyi commented on a change in pull request #351: Extend Iceberg with a way to overwrite files for eager updates/deletes URL: https://github.com/apache/incubator-iceberg/pull/351#discussion_r314236321 ## File path: api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java ## @@ -52,7 +52,7 @@ private MetricsEvalVisitor visitor() { return visitors.get(); } - InclusiveMetricsEvaluator(Schema schema, Expression unbound) { + public InclusiveMetricsEvaluator(Schema schema, Expression unbound) { Review comment: TODO: this is temporary until we figure out case sensitivity for metrics 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] aokolnychyi commented on issue #351: Extend Iceberg with a way to overwrite files for eager updates/deletes
aokolnychyi commented on issue #351: Extend Iceberg with a way to overwrite files for eager updates/deletes URL: https://github.com/apache/incubator-iceberg/pull/351#issuecomment-521580911 @rdblue What is our story on case sensitivity for metrics evaluators? We respect it in `FilteredManifest`, for example. However, `MergingSnapshotProducer` and `OverwriteData` use `StrictMetricsEvaluator` with `caseSensitive` always set to `true`. Do we want to expose a method on `PendingUpdate` to configure it? Any better idea? 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] aokolnychyi commented on issue #389: Add test cases
aokolnychyi commented on issue #389: Add test cases URL: https://github.com/apache/incubator-iceberg/pull/389#issuecomment-521594956 I actually didn't know we have `ScanSummary`. Do you use it for debugging purposes? 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] jerryshao commented on issue #370: Iceberg failed to work with embedded metastore, which is by default in Spark
jerryshao commented on issue #370: Iceberg failed to work with embedded metastore, which is by default in Spark URL: https://github.com/apache/incubator-iceberg/issues/370#issuecomment-521614634 Working on it, will submit a PR. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] aokolnychyi commented on issue #387: Optimize overwrite and delete commits.
aokolnychyi commented on issue #387: Optimize overwrite and delete commits. URL: https://github.com/apache/incubator-iceberg/pull/387#issuecomment-521615443 I think it is a substantial feature, can we add a couple of tests? 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] aokolnychyi commented on issue #387: Optimize overwrite and delete commits.
aokolnychyi commented on issue #387: Optimize overwrite and delete commits. URL: https://github.com/apache/incubator-iceberg/pull/387#issuecomment-521615928 Do we want to overload `deleteFile(DataFile)` in `StreamingDelete` to use this functionality? Right now, it will call `deleteFile(CharSequence)` under the hood. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] aokolnychyi commented on issue #387: Optimize overwrite and delete commits.
aokolnychyi commented on issue #387: Optimize overwrite and delete commits. URL: https://github.com/apache/incubator-iceberg/pull/387#issuecomment-521618913 Once this is merged, I'll update #351 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] aokolnychyi commented on a change in pull request #362: Support create and replace transactions in Catalog
aokolnychyi commented on a change in pull request #362: Support create and replace transactions in Catalog URL: https://github.com/apache/incubator-iceberg/pull/362#discussion_r314314329 ## File path: api/src/main/java/org/apache/iceberg/catalog/Catalog.java ## @@ -97,6 +98,42 @@ default Table createTable( return createTable(identifier, schema, PartitionSpec.unpartitioned(), null, null); } + /** + * Start a transaction to create a table. + * + * @param identifier a table identifier + * @param schema a schema + * @param spec a partition spec + * @param location a location for the table; leave null if unspecified + * @param properties a string map of table properties + * @return a {@link Transaction} to create the table + * @throws AlreadyExistsException if the table already exists + */ + Transaction newCreateTableTransaction( Review comment: I'll add them 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] aokolnychyi commented on a change in pull request #362: Support create and replace transactions in Catalog
aokolnychyi commented on a change in pull request #362: Support create and replace transactions in Catalog URL: https://github.com/apache/incubator-iceberg/pull/362#discussion_r314315329 ## File path: hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java ## @@ -0,0 +1,211 @@ +/* + * 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. + */ + +package org.apache.iceberg.hive; + +import com.google.common.collect.Maps; +import java.io.IOException; +import java.util.HashMap; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.types.Types; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; + +import static org.apache.iceberg.PartitionSpec.builderFor; +import static org.apache.iceberg.types.Types.NestedField.required; + +public class HiveCreateReplaceTableTest { + + private static final String DB_NAME = "hivedb"; + private static final String TABLE_NAME = "tbl"; + private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME); + private static final Schema SCHEMA = new Schema( + required(3, "id", Types.IntegerType.get()), + required(4, "data", Types.StringType.get()) + ); + private static final PartitionSpec SPEC = builderFor(SCHEMA) + .identity("id") + .build(); + + private static TestHiveMetastore metastore; + private static HiveMetaStoreClient metastoreClient; + private static HiveConf hiveConf; + private static HiveCatalog catalog; + + @Rule + public ExpectedException exceptionRule = ExpectedException.none(); + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + private String tableLocation; + + @BeforeClass + public static void startMetastore() throws Exception { +HiveCreateReplaceTableTest.metastore = new TestHiveMetastore(); +metastore.start(); +HiveCreateReplaceTableTest.hiveConf = metastore.hiveConf(); +HiveCreateReplaceTableTest.metastoreClient = new HiveMetaStoreClient(hiveConf); +String dbPath = metastore.getDatabasePath(DB_NAME); +Database db = new Database(DB_NAME, "description", dbPath, new HashMap<>()); +metastoreClient.createDatabase(db); +HiveCreateReplaceTableTest.catalog = new HiveCatalog(hiveConf); + } + + @AfterClass + public static void stopMetastore() { +catalog.close(); +HiveCreateReplaceTableTest.catalog = null; + +metastoreClient.close(); +HiveCreateReplaceTableTest.metastoreClient = null; + +metastore.stop(); +HiveCreateReplaceTableTest.metastore = null; + } + + @Before + public void createTableLocation() throws IOException { +tableLocation = temp.newFolder("hive-").getPath(); + } + + @After + public void cleanup() { +catalog.dropTable(TABLE_IDENTIFIER); + } + + @Test + public void testCreateTableTxn() { +Assert.assertFalse("Table should not exist", catalog.tableExists(TABLE_IDENTIFIER)); + +Transaction txn = catalog.newCreateTableTransaction( +TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap()); +txn.updateProperties() +.set("prop", "value") +.commit(); + +// verify the table is still not visible before the transaction is committed +Assert.assertFalse(catalog.tableExists(TABLE_IDENTIFIER)); + +txn.commitTransaction(); + +Table table = catalog.loadTable(TABLE_IDENTIFIER); +Assert.assertEquals("Table props should match", "value", table.properties().get("prop")); + } + + @Test + public void testCreateTableTxnTableCreatedConcurrently() { +exceptionRule.expect(RuntimeException.class); +
[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #351: Extend Iceberg with a way to overwrite files for eager updates/deletes
rdblue commented on a change in pull request #351: Extend Iceberg with a way to overwrite files for eager updates/deletes URL: https://github.com/apache/incubator-iceberg/pull/351#discussion_r314385635 ## File path: core/src/main/java/org/apache/iceberg/OverwriteData.java ## @@ -88,6 +121,40 @@ public OverwriteFiles validateAddedFiles() { } } +if (conflictingAppendsRowFilter != null) { + PartitionSpec spec = writeSpec(); + Expression inclusiveExpr = Projections.inclusive(spec).project(conflictingAppendsRowFilter); + Evaluator inclusive = new Evaluator(spec.partitionType(), inclusiveExpr); + + List newFiles = collectNewFiles(base); + for (DataFile newFile : newFiles) { +// we do partition-level conflict resolution right now +// we can enhance it by leveraging column stats and MetricsEvaluator +ValidationException.check( +!inclusive.eval(newFile.partition()), +"A conflicting file was appended that matches filter '%s': %s", +conflictingAppendsRowFilter, newFile.path()); + } +} + return super.apply(base); } + + private List collectNewFiles(TableMetadata meta) { +List newFiles = new ArrayList<>(); + +Long currentSnapshotId = meta.currentSnapshot() == null ? null : meta.currentSnapshot().snapshotId(); +while (currentSnapshotId != null && !currentSnapshotId.equals(readSnapshotId)) { + Snapshot currentSnapshot = meta.snapshot(currentSnapshotId); + + if (currentSnapshot == null) { +throw new ValidationException("Cannot find snapshot %d. Was it expired?", currentSnapshotId); Review comment: I agree that this would work correctly, but "Cannot find snapshot null" isn't a very helpful error message. I think it makes sense to catch that case earlier and have a better error. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] rdblue commented on issue #351: Extend Iceberg with a way to overwrite files for eager updates/deletes
rdblue commented on issue #351: Extend Iceberg with a way to overwrite files for eager updates/deletes URL: https://github.com/apache/incubator-iceberg/pull/351#issuecomment-521699751 I guess we should add a method to configure case sensitivity. We could also add a boolean flag to the methods where expressions are passed, or have case sensitive name variants? We should definitely handle case sensitivity, though. Not doing that is an oversight. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] rdblue commented on issue #389: Add test cases
rdblue commented on issue #389: Add test cases URL: https://github.com/apache/incubator-iceberg/pull/389#issuecomment-521700270 We use `ScanSummary` to provide partition-level summaries from our metastore. It's something that we could have put elsewhere, but I thought it was useful enough to go in Iceberg. We should probably move it over to use `FindFiles` at some point. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] rdblue merged pull request #389: Add test cases
rdblue merged pull request #389: Add test cases URL: https://github.com/apache/incubator-iceberg/pull/389 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] rdblue commented on issue #387: Optimize overwrite and delete commits.
rdblue commented on issue #387: Optimize overwrite and delete commits. URL: https://github.com/apache/incubator-iceberg/pull/387#issuecomment-521700796 @aokolnychyi, the existing tests cover correctness. This just speeds up the operations by ignoring manifests that can't match. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] rdblue merged pull request #385: Make original ManifestReader factory method public
rdblue merged pull request #385: Make original ManifestReader factory method public URL: https://github.com/apache/incubator-iceberg/pull/385 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] rdblue merged pull request #384: Add BaseCombinedScanTask.toString
rdblue merged pull request #384: Add BaseCombinedScanTask.toString URL: https://github.com/apache/incubator-iceberg/pull/384 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] rdblue merged pull request #377: Add FindFiles helper API
rdblue merged pull request #377: Add FindFiles helper API URL: https://github.com/apache/incubator-iceberg/pull/377 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] rdblue commented on issue #377: Add FindFiles helper API
rdblue commented on issue #377: Add FindFiles helper API URL: https://github.com/apache/incubator-iceberg/pull/377#issuecomment-521710811 Merging this. Thanks for reviewing, @xabriel! 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #374: Migrate spark table to iceberg table
rdblue commented on a change in pull request #374: Migrate spark table to iceberg table URL: https://github.com/apache/incubator-iceberg/pull/374#discussion_r314397922 ## File path: spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala ## @@ -297,5 +301,88 @@ object SparkTableUtil { ) } } + + def buildManifest(table: Table, +sparkDataFiles: Seq[SparkDataFile], +partitionSpec: PartitionSpec): ManifestFile = { +val outputFile = table.io + .newOutputFile(FileFormat.AVRO.addExtension("/tmp/" + UUID.randomUUID.toString)) +val writer = ManifestWriter.write(partitionSpec, outputFile) +try { + for (file <- sparkDataFiles) { +writer.add(file.toDataFile(partitionSpec)) + } +} finally { + writer.close() +} + +writer.toManifestFile + } + + def partitionToMap(partition: String): Map[String, String] = { +val map = new mutable.HashMap[String, String]() +val list = partition.split("/") +list.foreach { str => + val kv = str.split("=") + map.put(kv(0), kv(1)) +} + +map.toMap + } + + /** + * Migrate a spark table to a iceberg table. + * + * The migration uses the spark session to get table metadata. It assumes no + * operation is going on original table and target table and thus is not + * thread-safe. + * + * @param dbName the database name of the table to be migrated + * @param tableName the table to be migrated + * @param table the target table to migrate in + * + * @return table the target table + */ + def migrateSparkTable(dbName: String, tableName: String, table: Table): Table = { +val sparkSession = SparkSession.builder().getOrCreate() + +if (!sparkSession.catalog.tableExists(dbName, tableName)) { + throw new NoSuchTableException(s"Table $dbName.$tableName does not exist") +} + +val tableMetadata = sparkSession.sessionState.catalog. + getTableMetadata(new TableIdentifier(tableName, Some(dbName))) + +val format = tableMetadata.provider.getOrElse("none") +if (format != "avro" && format != "parquet" && format != "orc") { + throw new UnsupportedOperationException(s"Unsupported format: $format") +} + +val location = tableMetadata.location.toString +val partitionSpec = SparkSchemaUtil.specForTable(sparkSession, s"$dbName.$tableName") + +val fastAppender = table.newFastAppend() Review comment: I don't think there is a reason to use fast appends. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #374: Migrate spark table to iceberg table
rdblue commented on a change in pull request #374: Migrate spark table to iceberg table URL: https://github.com/apache/incubator-iceberg/pull/374#discussion_r314399396 ## File path: spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala ## @@ -297,5 +301,88 @@ object SparkTableUtil { ) } } + + def buildManifest(table: Table, +sparkDataFiles: Seq[SparkDataFile], +partitionSpec: PartitionSpec): ManifestFile = { +val outputFile = table.io + .newOutputFile(FileFormat.AVRO.addExtension("/tmp/" + UUID.randomUUID.toString)) +val writer = ManifestWriter.write(partitionSpec, outputFile) +try { + for (file <- sparkDataFiles) { +writer.add(file.toDataFile(partitionSpec)) + } +} finally { + writer.close() +} + +writer.toManifestFile + } + + def partitionToMap(partition: String): Map[String, String] = { +val map = new mutable.HashMap[String, String]() +val list = partition.split("/") +list.foreach { str => + val kv = str.split("=") + map.put(kv(0), kv(1)) +} + +map.toMap + } + + /** + * Migrate a spark table to a iceberg table. + * + * The migration uses the spark session to get table metadata. It assumes no + * operation is going on original table and target table and thus is not + * thread-safe. + * + * @param dbName the database name of the table to be migrated + * @param tableName the table to be migrated + * @param table the target table to migrate in + * + * @return table the target table + */ + def migrateSparkTable(dbName: String, tableName: String, table: Table): Table = { +val sparkSession = SparkSession.builder().getOrCreate() + +if (!sparkSession.catalog.tableExists(dbName, tableName)) { + throw new NoSuchTableException(s"Table $dbName.$tableName does not exist") +} + +val tableMetadata = sparkSession.sessionState.catalog. + getTableMetadata(new TableIdentifier(tableName, Some(dbName))) + +val format = tableMetadata.provider.getOrElse("none") Review comment: This should convert any Hive table, not just Spark DataSource tables. That's why the `listPartition` method supports a format for each partition. Use `partitionDF` to get a dataframe with each partition and format. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #374: Migrate spark table to iceberg table
rdblue commented on a change in pull request #374: Migrate spark table to iceberg table URL: https://github.com/apache/incubator-iceberg/pull/374#discussion_r314399797 ## File path: spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala ## @@ -297,5 +301,88 @@ object SparkTableUtil { ) } } + + def buildManifest(table: Table, +sparkDataFiles: Seq[SparkDataFile], +partitionSpec: PartitionSpec): ManifestFile = { +val outputFile = table.io + .newOutputFile(FileFormat.AVRO.addExtension("/tmp/" + UUID.randomUUID.toString)) +val writer = ManifestWriter.write(partitionSpec, outputFile) +try { + for (file <- sparkDataFiles) { +writer.add(file.toDataFile(partitionSpec)) + } +} finally { + writer.close() +} + +writer.toManifestFile + } + + def partitionToMap(partition: String): Map[String, String] = { +val map = new mutable.HashMap[String, String]() +val list = partition.split("/") +list.foreach { str => + val kv = str.split("=") + map.put(kv(0), kv(1)) +} + +map.toMap + } + + /** + * Migrate a spark table to a iceberg table. + * + * The migration uses the spark session to get table metadata. It assumes no + * operation is going on original table and target table and thus is not + * thread-safe. + * + * @param dbName the database name of the table to be migrated + * @param tableName the table to be migrated + * @param table the target table to migrate in + * + * @return table the target table + */ + def migrateSparkTable(dbName: String, tableName: String, table: Table): Table = { +val sparkSession = SparkSession.builder().getOrCreate() + +if (!sparkSession.catalog.tableExists(dbName, tableName)) { + throw new NoSuchTableException(s"Table $dbName.$tableName does not exist") +} + +val tableMetadata = sparkSession.sessionState.catalog. + getTableMetadata(new TableIdentifier(tableName, Some(dbName))) + +val format = tableMetadata.provider.getOrElse("none") +if (format != "avro" && format != "parquet" && format != "orc") { + throw new UnsupportedOperationException(s"Unsupported format: $format") +} + +val location = tableMetadata.location.toString +val partitionSpec = SparkSchemaUtil.specForTable(sparkSession, s"$dbName.$tableName") + +val fastAppender = table.newFastAppend() + +val partitions = sparkSession.sessionState.catalog.externalCatalog + .listPartitionNames(dbName, tableName) +if (partitions.isEmpty) { + val dataFiles = SparkTableUtil.listPartition(Map.empty[String, String], location, format) + fastAppender.appendManifest(buildManifest(table, dataFiles, PartitionSpec.unpartitioned)) +} else { + // Retrieve data files according to partition. result = [[datafiles], [datafiles]] + val dataFiles = partitions.map { e => +SparkTableUtil.listPartition(partitionToMap(e), location + "/" + e, format) + } + + // Append manifest for each partition + dataFiles.foreach { partition => +fastAppender.appendManifest(buildManifest(table, partition, partitionSpec)) + } +} + +fastAppender.apply() Review comment: There's no need to call apply. Just commit. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #374: Migrate spark table to iceberg table
rdblue commented on a change in pull request #374: Migrate spark table to iceberg table URL: https://github.com/apache/incubator-iceberg/pull/374#discussion_r314399700 ## File path: spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala ## @@ -297,5 +301,88 @@ object SparkTableUtil { ) } } + + def buildManifest(table: Table, +sparkDataFiles: Seq[SparkDataFile], +partitionSpec: PartitionSpec): ManifestFile = { +val outputFile = table.io + .newOutputFile(FileFormat.AVRO.addExtension("/tmp/" + UUID.randomUUID.toString)) +val writer = ManifestWriter.write(partitionSpec, outputFile) +try { + for (file <- sparkDataFiles) { +writer.add(file.toDataFile(partitionSpec)) + } +} finally { + writer.close() +} + +writer.toManifestFile + } + + def partitionToMap(partition: String): Map[String, String] = { +val map = new mutable.HashMap[String, String]() +val list = partition.split("/") +list.foreach { str => + val kv = str.split("=") + map.put(kv(0), kv(1)) +} + +map.toMap + } + + /** + * Migrate a spark table to a iceberg table. + * + * The migration uses the spark session to get table metadata. It assumes no + * operation is going on original table and target table and thus is not + * thread-safe. + * + * @param dbName the database name of the table to be migrated + * @param tableName the table to be migrated + * @param table the target table to migrate in + * + * @return table the target table + */ + def migrateSparkTable(dbName: String, tableName: String, table: Table): Table = { +val sparkSession = SparkSession.builder().getOrCreate() + +if (!sparkSession.catalog.tableExists(dbName, tableName)) { + throw new NoSuchTableException(s"Table $dbName.$tableName does not exist") +} + +val tableMetadata = sparkSession.sessionState.catalog. + getTableMetadata(new TableIdentifier(tableName, Some(dbName))) + +val format = tableMetadata.provider.getOrElse("none") +if (format != "avro" && format != "parquet" && format != "orc") { + throw new UnsupportedOperationException(s"Unsupported format: $format") +} + +val location = tableMetadata.location.toString +val partitionSpec = SparkSchemaUtil.specForTable(sparkSession, s"$dbName.$tableName") + +val fastAppender = table.newFastAppend() + +val partitions = sparkSession.sessionState.catalog.externalCatalog + .listPartitionNames(dbName, tableName) Review comment: This should use `partitionDF` and call `listPartition` in parallel for large tables. See the example notebook: https://github.com/apache/incubator-iceberg/blob/master/examples/Convert%20table%20to%20Iceberg.ipynb 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] linxingyuan1102 commented on issue #227: ORC column map fix
linxingyuan1102 commented on issue #227: ORC column map fix URL: https://github.com/apache/incubator-iceberg/pull/227#issuecomment-521736621 FYI, I prototyped ID-based column mapping for Presto in prestosql/presto#1290, using the type annotations introduced to ORC. So the type annotation is a key/value pair. I think we will need to agree on the key value and specify it in the spec. cc: @rdblue @edgarRd @omalley Do we have any update on releasing 1.6? 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] rdblue commented on issue #227: ORC column map fix
rdblue commented on issue #227: ORC column map fix URL: https://github.com/apache/incubator-iceberg/pull/227#issuecomment-521738363 Thanks for the update, @linxingyuan1102. It would be great to have your help getting the spec updated. In the end, I think Presto should depend on Iceberg to convert an ORC file's schema to an Iceberg schema so that we can more easily guarantee compatibility, but in theory we should be able to re-implement the conversion to a schema with IDs. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] aokolnychyi commented on issue #351: Extend Iceberg with a way to overwrite files for eager updates/deletes
aokolnychyi commented on issue #351: Extend Iceberg with a way to overwrite files for eager updates/deletes URL: https://github.com/apache/incubator-iceberg/pull/351#issuecomment-521755450 My first guess was to use a separate method as it is probably more descriptive than passing a boolean parameter (as discussed before, no way to pass named variables in Java). I would be also ok to provide name variants. Which option would you prefer personally? 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] edgarRd commented on issue #227: ORC column map fix
edgarRd commented on issue #227: ORC column map fix URL: https://github.com/apache/incubator-iceberg/pull/227#issuecomment-521789477 @linxingyuan1102 I agree, we need to specify the annotations for ORC column mapping in the spec. I have a proposal in https://github.com/apache/incubator-iceberg/pull/227/files#diff-116b0afb33e66021cf000896fde94817L492 - although this applies to the serialization of all column mappings, I suppose that for the new Type attributes in ORC it would be per column. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] manishmalhotrawork opened a new pull request #393: include column_sizes in stats columns
manishmalhotrawork opened a new pull request #393: include column_sizes in stats columns URL: https://github.com/apache/incubator-iceberg/pull/393 this is for Issue #269 My understand is that `column_sizes` was calculated by `ParquetUtil.footerMetrics` or `ParquetUtil.fileMetrics` but calling `table.newScan().includeColumnStats()` is still not populating. @aokolnychyi if you want to review, please. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314505113 ## File path: api/src/main/java/org/apache/iceberg/expressions/LiteralSet.java ## @@ -0,0 +1,212 @@ +/* + * 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. + */ + +package org.apache.iceberg.expressions; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableSet; +import java.io.Serializable; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Objects; +import java.util.Set; +import org.apache.iceberg.types.Comparators; + +/** + * Represents a set of literal values in an IN or NOT_IN predicate + * @param The Java type of the value, which can be wrapped by a {@link Literal} + */ +public class LiteralSet implements Set, Serializable { + private final Set literals; + + @SuppressWarnings("unchecked") + LiteralSet(Set> lits) { +Preconditions.checkArgument(lits == null || lits.size() > 1, +"The input literal set must include more than 1 element."); +literals = ImmutableSet.builder().addAll( +lits.stream().map( +lit -> { + if (lit instanceof Literals.StringLiteral) { +return (T) new CharSeqWrapper((CharSequence) lit.value()); + } else { +return lit.value(); + } +} +).iterator()).build(); + } + + @Override + public String toString() { +Iterator it = literals.iterator(); Review comment: 👌 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] aokolnychyi commented on issue #393: include column_sizes in stats columns
aokolnychyi commented on issue #393: include column_sizes in stats columns URL: https://github.com/apache/incubator-iceberg/pull/393#issuecomment-521808846 LGTM. Thanks, @manishmalhotrawork! 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] waterlx opened a new issue #394: Wrong URL of "Iceberg API" on Spark user docs
waterlx opened a new issue #394: Wrong URL of "Iceberg API" on Spark user docs URL: https://github.com/apache/incubator-iceberg/issues/394 On the page of https://iceberg.incubator.apache.org/spark/, in "Spark 2.4 is limited to reading and writing existing Iceberg tables. Use the Iceberg API to create Iceberg tables", URL of "Iceberg API" is pointing to "https://iceberg.incubator.apache.org/spark/api";, which does not exist 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314583647 ## File path: api/src/main/java/org/apache/iceberg/expressions/LiteralSet.java ## @@ -0,0 +1,212 @@ +/* + * 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. + */ + +package org.apache.iceberg.expressions; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableSet; +import java.io.Serializable; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Objects; +import java.util.Set; +import org.apache.iceberg.types.Comparators; + +/** + * Represents a set of literal values in an IN or NOT_IN predicate + * @param The Java type of the value, which can be wrapped by a {@link Literal} + */ +public class LiteralSet implements Set, Serializable { + private final Set literals; + + @SuppressWarnings("unchecked") + LiteralSet(Set> lits) { +Preconditions.checkArgument(lits == null || lits.size() > 1, +"The input literal set must include more than 1 element."); +literals = ImmutableSet.builder().addAll( +lits.stream().map( +lit -> { + if (lit instanceof Literals.StringLiteral) { +return (T) new CharSeqWrapper((CharSequence) lit.value()); + } else { +return lit.value(); + } +} +).iterator()).build(); + } + + @Override + public String toString() { +Iterator it = literals.iterator(); +if (!it.hasNext()) { + return "{}"; +} + +StringBuilder sb = new StringBuilder(); +sb.append('{'); +while (true) { + sb.append(it.next()); + if (!it.hasNext()) { +return sb.append('}').toString(); + } + sb.append(',').append(' '); +} + } + + @Override + @SuppressWarnings("unchecked") + public boolean equals(Object other) { Review comment: Thanks for the comments. That has been used in `TestExpressionSerialization` to check the equality of two predicates. As that is the only place using it, I am going to move it inside `TestExpressionSerialization` and remove `equals` and `hashCode` from the `LiteralSet`. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314583886 ## File path: api/src/main/java/org/apache/iceberg/expressions/LiteralSet.java ## @@ -0,0 +1,212 @@ +/* + * 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. + */ + +package org.apache.iceberg.expressions; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableSet; +import java.io.Serializable; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Objects; +import java.util.Set; +import org.apache.iceberg.types.Comparators; + +/** + * Represents a set of literal values in an IN or NOT_IN predicate + * @param The Java type of the value, which can be wrapped by a {@link Literal} + */ +public class LiteralSet implements Set, Serializable { + private final Set literals; + + @SuppressWarnings("unchecked") + LiteralSet(Set> lits) { +Preconditions.checkArgument(lits == null || lits.size() > 1, +"The input literal set must include more than 1 element."); +literals = ImmutableSet.builder().addAll( +lits.stream().map( +lit -> { + if (lit instanceof Literals.StringLiteral) { +return (T) new CharSeqWrapper((CharSequence) lit.value()); + } else { +return lit.value(); + } +} +).iterator()).build(); + } + + @Override + public String toString() { +Iterator it = literals.iterator(); +if (!it.hasNext()) { + return "{}"; +} + +StringBuilder sb = new StringBuilder(); +sb.append('{'); +while (true) { + sb.append(it.next()); + if (!it.hasNext()) { +return sb.append('}').toString(); + } + sb.append(',').append(' '); +} + } + + @Override + @SuppressWarnings("unchecked") + public boolean equals(Object other) { +if (this == other) { + return true; +} +if (other == null || getClass() != other.getClass()) { + return false; +} +LiteralSet that = (LiteralSet) other; +return literals.equals(that.literals); + } + + @Override + public int hashCode() { +return Objects.hashCode(literals); + } + + @Override + public boolean contains(Object object) { +return literals.contains(object); + } + + @Override + public int size() { +return literals.size(); + } + + @Override + public boolean isEmpty() { +return literals.isEmpty(); + } + + @Override + public Iterator iterator() { +return literals.iterator(); + } + + @Override + public Object[] toArray() { +return literals.toArray(); + } + + @Override + public X[] toArray(X[] a) { +return literals.toArray(a); + } + + @Override + public boolean containsAll(Collection c) { +return literals.containsAll(c); + } + + @Override + public boolean add(T t) { +throw new UnsupportedOperationException(); + } + + @Override + public boolean remove(Object o) { +throw new UnsupportedOperationException(); + } + + + @Override + public boolean addAll(Collection c) { +throw new UnsupportedOperationException(); + } + + @Override + public boolean retainAll(Collection c) { +throw new UnsupportedOperationException(); + } + + @Override + public boolean removeAll(Collection c) { +throw new UnsupportedOperationException(); + } + + @Override + public void clear() { +throw new UnsupportedOperationException(); + } + + static class CharSeqWrapper implements CharSequence, Serializable { Review comment: Thanks for the suggestion! Will remove `CharSeqWrapper` and reuse the `CharSequenceWrapper` instead. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org F
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314583939 ## File path: api/src/main/java/org/apache/iceberg/expressions/LiteralSet.java ## @@ -0,0 +1,212 @@ +/* + * 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. + */ + +package org.apache.iceberg.expressions; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableSet; +import java.io.Serializable; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Objects; +import java.util.Set; +import org.apache.iceberg.types.Comparators; + +/** + * Represents a set of literal values in an IN or NOT_IN predicate + * @param The Java type of the value, which can be wrapped by a {@link Literal} + */ +public class LiteralSet implements Set, Serializable { + private final Set literals; + + @SuppressWarnings("unchecked") + LiteralSet(Set> lits) { +Preconditions.checkArgument(lits == null || lits.size() > 1, +"The input literal set must include more than 1 element."); +literals = ImmutableSet.builder().addAll( +lits.stream().map( +lit -> { + if (lit instanceof Literals.StringLiteral) { +return (T) new CharSeqWrapper((CharSequence) lit.value()); + } else { +return lit.value(); + } +} +).iterator()).build(); + } + + @Override + public String toString() { +Iterator it = literals.iterator(); +if (!it.hasNext()) { + return "{}"; +} + +StringBuilder sb = new StringBuilder(); +sb.append('{'); +while (true) { + sb.append(it.next()); + if (!it.hasNext()) { +return sb.append('}').toString(); + } + sb.append(',').append(' '); +} + } + + @Override + @SuppressWarnings("unchecked") + public boolean equals(Object other) { +if (this == other) { + return true; +} +if (other == null || getClass() != other.getClass()) { + return false; +} +LiteralSet that = (LiteralSet) other; +return literals.equals(that.literals); + } + + @Override + public int hashCode() { +return Objects.hashCode(literals); + } + + @Override + public boolean contains(Object object) { +return literals.contains(object); + } + + @Override + public int size() { +return literals.size(); + } + + @Override + public boolean isEmpty() { +return literals.isEmpty(); + } + + @Override + public Iterator iterator() { +return literals.iterator(); + } + + @Override + public Object[] toArray() { +return literals.toArray(); + } + + @Override + public X[] toArray(X[] a) { +return literals.toArray(a); + } + + @Override + public boolean containsAll(Collection c) { +return literals.containsAll(c); + } + + @Override + public boolean add(T t) { +throw new UnsupportedOperationException(); + } + + @Override + public boolean remove(Object o) { +throw new UnsupportedOperationException(); + } + + + @Override + public boolean addAll(Collection c) { +throw new UnsupportedOperationException(); + } + + @Override + public boolean retainAll(Collection c) { +throw new UnsupportedOperationException(); + } + + @Override + public boolean removeAll(Collection c) { +throw new UnsupportedOperationException(); + } + + @Override + public void clear() { +throw new UnsupportedOperationException(); + } + + static class CharSeqWrapper implements CharSequence, Serializable { +private static final Comparator CMP = + Comparators.nullsFirst().thenComparing(Comparators.charSequences()); + +private final CharSequence chars; + +CharSeqWrapper(CharSequence charSequence) { + Preconditions.checkNotNull(charSequence, "String literal values cannot be null"); + this.chars = charSequence; +} + +@Override +public int length() { + return chars.length(); +} + +@Override +public char charAt(int index) { + return chars.charAt(index); +} + +@Override +public CharSequence subSequence(int start, int end) { + return chars.su
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314584147 ## File path: api/src/main/java/org/apache/iceberg/expressions/ExpressionVisitors.java ## @@ -89,12 +93,12 @@ public R or(R leftResult, R rightResult) { return null; } -public R in(BoundReference ref, Literal lit) { - return null; +public R in(BoundReference ref, LiteralSet literalSet) { Review comment: Yep, thanks for the comments. Updated them accordingly. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314584436 ## File path: api/src/main/java/org/apache/iceberg/expressions/LiteralSet.java ## @@ -0,0 +1,212 @@ +/* + * 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. + */ + +package org.apache.iceberg.expressions; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableSet; +import java.io.Serializable; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Objects; +import java.util.Set; +import org.apache.iceberg.types.Comparators; + +/** + * Represents a set of literal values in an IN or NOT_IN predicate + * @param The Java type of the value, which can be wrapped by a {@link Literal} + */ +public class LiteralSet implements Set, Serializable { + private final Set literals; + + @SuppressWarnings("unchecked") + LiteralSet(Set> lits) { +Preconditions.checkArgument(lits == null || lits.size() > 1, +"The input literal set must include more than 1 element."); +literals = ImmutableSet.builder().addAll( +lits.stream().map( +lit -> { + if (lit instanceof Literals.StringLiteral) { +return (T) new CharSeqWrapper((CharSequence) lit.value()); + } else { +return lit.value(); + } +} +).iterator()).build(); + } + + @Override + public String toString() { +Iterator it = literals.iterator(); +if (!it.hasNext()) { + return "{}"; +} + +StringBuilder sb = new StringBuilder(); +sb.append('{'); +while (true) { + sb.append(it.next()); + if (!it.hasNext()) { +return sb.append('}').toString(); + } + sb.append(',').append(' '); +} + } + + @Override + @SuppressWarnings("unchecked") + public boolean equals(Object other) { +if (this == other) { + return true; +} +if (other == null || getClass() != other.getClass()) { + return false; +} +LiteralSet that = (LiteralSet) other; +return literals.equals(that.literals); + } + + @Override + public int hashCode() { +return Objects.hashCode(literals); + } + + @Override + public boolean contains(Object object) { +return literals.contains(object); + } + + @Override + public int size() { +return literals.size(); + } + + @Override + public boolean isEmpty() { +return literals.isEmpty(); + } + + @Override + public Iterator iterator() { +return literals.iterator(); + } + + @Override + public Object[] toArray() { +return literals.toArray(); + } + + @Override + public X[] toArray(X[] a) { +return literals.toArray(a); Review comment: Yep, It should. As `toArray` methods are not used, I will make them to throw exceptions. Then, in the iterator, I added the logic to unwrap the `CharSequence`. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314585257 ## File path: api/src/main/java/org/apache/iceberg/expressions/LiteralSet.java ## @@ -0,0 +1,212 @@ +/* + * 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. + */ + +package org.apache.iceberg.expressions; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableSet; +import java.io.Serializable; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Objects; +import java.util.Set; +import org.apache.iceberg.types.Comparators; + +/** + * Represents a set of literal values in an IN or NOT_IN predicate + * @param The Java type of the value, which can be wrapped by a {@link Literal} + */ +public class LiteralSet implements Set, Serializable { + private final Set literals; + + @SuppressWarnings("unchecked") + LiteralSet(Set> lits) { +Preconditions.checkArgument(lits == null || lits.size() > 1, +"The input literal set must include more than 1 element."); +literals = ImmutableSet.builder().addAll( +lits.stream().map( +lit -> { + if (lit instanceof Literals.StringLiteral) { +return (T) new CharSeqWrapper((CharSequence) lit.value()); + } else { +return lit.value(); + } +} +).iterator()).build(); + } + + @Override + public String toString() { +Iterator it = literals.iterator(); +if (!it.hasNext()) { + return "{}"; +} + +StringBuilder sb = new StringBuilder(); +sb.append('{'); +while (true) { + sb.append(it.next()); + if (!it.hasNext()) { +return sb.append('}').toString(); + } + sb.append(',').append(' '); +} + } + + @Override + @SuppressWarnings("unchecked") + public boolean equals(Object other) { +if (this == other) { + return true; +} +if (other == null || getClass() != other.getClass()) { + return false; +} +LiteralSet that = (LiteralSet) other; +return literals.equals(that.literals); + } + + @Override + public int hashCode() { +return Objects.hashCode(literals); + } + + @Override + public boolean contains(Object object) { +return literals.contains(object); + } + + @Override + public int size() { +return literals.size(); + } + + @Override + public boolean isEmpty() { +return literals.isEmpty(); + } + + @Override + public Iterator iterator() { +return literals.iterator(); + } + + @Override + public Object[] toArray() { +return literals.toArray(); + } + + @Override + public X[] toArray(X[] a) { +return literals.toArray(a); + } + + @Override + public boolean containsAll(Collection c) { +return literals.containsAll(c); + } + + @Override + public boolean add(T t) { +throw new UnsupportedOperationException(); Review comment: Thanks for the suggestions. Added the descriptions. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314585643 ## File path: api/src/main/java/org/apache/iceberg/expressions/ManifestEvaluator.java ## @@ -245,12 +245,12 @@ public Boolean or(Boolean leftResult, Boolean rightResult) { } @Override -public Boolean in(BoundReference ref, Literal lit) { +public Boolean in(BoundReference ref, LiteralSet literalSet) { return ROWS_MIGHT_MATCH; Review comment: @rdblue It is probably better to be implemented in the next PR, which I am going to work on it after this one. Please let me know your thoughts. thanks. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314585849 ## File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java ## @@ -285,12 +284,12 @@ public Boolean or(Boolean leftResult, Boolean rightResult) { } @Override -public Boolean in(BoundReference ref, Literal lit) { +public Boolean in(BoundReference ref, LiteralSet literalSet) { return ROWS_MIGHT_MATCH; Review comment: Yep, I agree. I think It is probably better to be implemented in the next PR, which I am going to work on it after this one. Please let me know your thoughts. thanks. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314586632 ## File path: api/src/main/java/org/apache/iceberg/expressions/Expressions.java ## @@ -109,16 +111,46 @@ public static Expression not(Expression child) { return new UnboundPredicate<>(Expression.Operation.STARTS_WITH, ref(name), value); } + public static UnboundPredicate in(String name, T value, T... values) { +return predicate(Operation.IN, name, +Stream.concat(Stream.of(value), Stream.of(values)) +.map(Literals::from).collect(Collectors.toSet())); + } + + public static UnboundPredicate notIn(String name, T value, T... values) { Review comment: @rdblue Thanks for the comment, I can make it to be `T...` here. I still think it may be better to throw exception instead of return `alwaysTrue` because that will force the method to return `Expression`. When users use it, they have to explicitly cast it to the concrete class, which makes it not-easy to use. Please let me know your thoughts. thanks. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314588864 ## File path: api/src/main/java/org/apache/iceberg/expressions/UnboundPredicate.java ## @@ -125,13 +154,38 @@ public Expression bind(Types.StructType struct, boolean caseSensitive) { case LT_EQ: case EQ: return Expressions.alwaysFalse(); -//case IN: -// break; -//case NOT_IN: -// break; } } return new BoundPredicate<>(op(), new BoundReference<>(field.fieldId(), -schema.accessorForField(field.fieldId())), lit); +schema.accessorForField(field.fieldId())), lit); + } + + @SuppressWarnings("unchecked") + private Expression bindInOperation(Types.NestedField field, Schema schema) { +final Set> lits = literalSet().stream().map( +val -> { + if (val instanceof LiteralSet.CharSeqWrapper) { +val = (T) ((LiteralSet.CharSeqWrapper) val).unWrap(); + } + Literal lit = Literals.from(val).to(field.type()); + if (lit == null) { +throw new ValidationException(String.format( +"Invalid value for comparison inclusive type %s: %s (%s)", +field.type(), val, val.getClass().getName())); + } + return lit; +}) +.filter(l -> l != Literals.aboveMax() && l != Literals.belowMin()) +.collect(Collectors.toSet()); Review comment: @rdblue The main reason to make it to be a `Set` is for deduplication. For example, ``` StructType struct = StructType.of(required(15, "d", Types.DecimalType.of(9, 2))); UnboundPredicate unbound = Expressions.in("d", 12.40, 12.401, 12.402); Expression expr = unbound.bind(struct); // expr should be `BoundPredicate` with `EQ` op. ``` So here, the size of literalSet might shrink due to the type conversion. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314589687 ## File path: api/src/main/java/org/apache/iceberg/expressions/Predicate.java ## @@ -19,15 +19,49 @@ package org.apache.iceberg.expressions; +import com.google.common.base.Preconditions; +import java.util.Set; + public abstract class Predicate implements Expression { private final Operation op; private final R ref; private final Literal literal; + private final LiteralSet literalSet; Review comment: Thanks for the comments. Updated the classes accordingly. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314589849 ## File path: api/src/main/java/org/apache/iceberg/expressions/UnboundPredicate.java ## @@ -125,13 +154,38 @@ public Expression bind(Types.StructType struct, boolean caseSensitive) { case LT_EQ: case EQ: return Expressions.alwaysFalse(); -//case IN: -// break; -//case NOT_IN: -// break; } } return new BoundPredicate<>(op(), new BoundReference<>(field.fieldId(), -schema.accessorForField(field.fieldId())), lit); +schema.accessorForField(field.fieldId())), lit); + } + + @SuppressWarnings("unchecked") + private Expression bindInOperation(Types.NestedField field, Schema schema) { +final Set> lits = literalSet().stream().map( +val -> { + if (val instanceof LiteralSet.CharSeqWrapper) { +val = (T) ((LiteralSet.CharSeqWrapper) val).unWrap(); + } + Literal lit = Literals.from(val).to(field.type()); Review comment: Sure, I will change 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314586632 ## File path: api/src/main/java/org/apache/iceberg/expressions/Expressions.java ## @@ -109,16 +111,46 @@ public static Expression not(Expression child) { return new UnboundPredicate<>(Expression.Operation.STARTS_WITH, ref(name), value); } + public static UnboundPredicate in(String name, T value, T... values) { +return predicate(Operation.IN, name, +Stream.concat(Stream.of(value), Stream.of(values)) +.map(Literals::from).collect(Collectors.toSet())); + } + + public static UnboundPredicate notIn(String name, T value, T... values) { Review comment: @rdblue Thanks for the comment, I will make it to be `T...` here. I still think it may be better to throw exception instead of return `alwaysTrue` because that will force the method to return `Expression`. When users use it, they have to explicitly cast it to the concrete class, which makes it not-easy to use. Please let me know your thoughts. thanks. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] jun-he commented on a change in pull request #357: Add in and not in predicates
jun-he commented on a change in pull request #357: Add in and not in predicates URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r314591877 ## File path: api/src/main/java/org/apache/iceberg/expressions/Literals.java ## @@ -109,6 +110,26 @@ public T value() { public String toString() { return String.valueOf(value); } + +@Override +@SuppressWarnings("unchecked") +public boolean equals(Object other) { Review comment: Yes, it is used for deduplication. Additionally, it might be better to override `equals` to be consistent with the behavior of the comparison. 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] waterlx commented on issue #394: Wrong URL of "Iceberg API" on Spark user docs
waterlx commented on issue #394: Wrong URL of "Iceberg API" on Spark user docs URL: https://github.com/apache/incubator-iceberg/issues/394#issuecomment-521903689 @rdblue I am not sure if the website is not deployed after [issue 333](https://github.com/apache/incubator-iceberg/pull/333) is merged. Would you please have a check at your convenience? 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org
[GitHub] [incubator-iceberg] waterlx edited a comment on issue #394: Wrong URL of "Iceberg API" on Spark user docs
waterlx edited a comment on issue #394: Wrong URL of "Iceberg API" on Spark user docs URL: https://github.com/apache/incubator-iceberg/issues/394#issuecomment-521903689 @rdblue I am not sure if the website is not deployed after #333 is merged. Would you please have a check at your convenience? 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services - To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org