[GitHub] [incubator-iceberg] aokolnychyi commented on a change in pull request #351: Extend Iceberg with a way to overwrite files for eager updates/deletes

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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.

2019-08-15 Thread GitBox
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.

2019-08-15 Thread GitBox
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.

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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.

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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

2019-08-15 Thread GitBox
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