[GitHub] [incubator-iceberg] chenjunjiedada commented on a change in pull request #374: Migrate spark table to iceberg table

2019-08-24 Thread GitBox
chenjunjiedada commented on a change in pull request #374: Migrate spark table 
to iceberg table
URL: https://github.com/apache/incubator-iceberg/pull/374#discussion_r317379934
 
 

 ##
 File path: spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala
 ##
 @@ -297,5 +301,81 @@ object SparkTableUtil {
   )
 }
   }
+
+  private 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) {
 
 Review comment:
   OK, will update.


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] chenjunjiedada commented on a change in pull request #374: Migrate spark table to iceberg table

2019-08-24 Thread GitBox
chenjunjiedada commented on a change in pull request #374: Migrate spark table 
to iceberg table
URL: https://github.com/apache/incubator-iceberg/pull/374#discussion_r317379931
 
 

 ##
 File path: spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala
 ##
 @@ -297,5 +301,81 @@ object SparkTableUtil {
   )
 }
   }
+
+  private 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
+  }
+
+  /**
+   * Import a spark table to a iceberg table.
+   *
+   * The import 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 source the database name of the table to be import
+   * @param location the location used to store table metadata
+   *
+   * @return table the imported table
+   */
+  def importSparkTable(source: TableIdentifier, location: String): Table = {
+val sparkSession = SparkSession.builder().getOrCreate()
+import sparkSession.sqlContext.implicits._
+
+val dbName = source.database.getOrElse("default")
+val tableName = source.table
+
+if (!sparkSession.catalog.tableExists(dbName, tableName)) {
+  throw new NoSuchTableException(s"Table $dbName.$tableName does not 
exist")
+}
+
+val partitionSpec = SparkSchemaUtil.specForTable(sparkSession, 
s"$dbName.$tableName")
+val conf = sparkSession.sparkContext.hadoopConfiguration
+val tables = new HadoopTables(conf)
+val schema = SparkSchemaUtil.schemaForTable(sparkSession, 
s"$dbName.$tableName")
+val table = tables.create(schema, partitionSpec, ImmutableMap.of(), 
location)
+val appender = table.newAppend()
+
+if (partitionSpec == PartitionSpec.unpartitioned) {
 
 Review comment:
   Yes, here is the exception:
   ```
   org.apache.hadoop.hive.ql.metadata.HiveException: Table item is not a 
partitioned table
 at org.apache.hadoop.hive.ql.metadata.Hive.getPartitions(Hive.java:2123)
 at org.apache.hadoop.hive.ql.metadata.Hive.getPartitions(Hive.java:2156)
 at 
org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getPartitions$1.apply(HiveClientImpl.scala:670)
 at 
org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$getPartitions$1.apply(HiveClientImpl.scala:662)
 at 
org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$withHiveState$1.apply(HiveClientImpl.scala:275)
 at 
org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:213)
 at 
org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:212)
 at 
org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:258)
 at 
org.apache.spark.sql.hive.client.HiveClientImpl.getPartitions(HiveClientImpl.scala:662)
 at 
org.apache.spark.sql.hive.client.HiveClient$class.getPartitions(HiveClient.scala:210)
 at 
org.apache.spark.sql.hive.client.HiveClientImpl.getPartitions(HiveClientImpl.scala:83)
 at org.apache.iceberg.spark.hacks.Hive.partitions(Hive.java:48)
 at 
org.apache.iceberg.spark.SparkTableUtil$.partitionDF(SparkTableUtil.scala:56)
   ```
   
   If that is not expected, I can try to fix that first


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] chenjunjiedada commented on a change in pull request #374: Migrate spark table to iceberg table

2019-08-24 Thread GitBox
chenjunjiedada commented on a change in pull request #374: Migrate spark table 
to iceberg table
URL: https://github.com/apache/incubator-iceberg/pull/374#discussion_r317379704
 
 

 ##
 File path: hive/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java
 ##
 @@ -111,6 +111,7 @@ private HiveConf newHiveConf(int port) {
 HiveConf newHiveConf = new HiveConf(new Configuration(), 
TestHiveMetastore.class);
 newHiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, 
"thrift://localhost:" + port);
 newHiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, "file:" + 
hiveLocalDir.getAbsolutePath());
+newHiveConf.set(HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname, 
"false");
 
 Review comment:
   WARN org.apache.hadoop.hive.metastore.ObjectStore - Direct SQL failed, 
falling back to ORM
   java.lang.ClassCastException: org.apache.derby.impl.jdbc.EmbedClob cannot be 
cast to java.lang.String.
   
   


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] chenjunjiedada commented on a change in pull request #374: Migrate spark table to iceberg table

2019-08-24 Thread GitBox
chenjunjiedada commented on a change in pull request #374: Migrate spark table 
to iceberg table
URL: https://github.com/apache/incubator-iceberg/pull/374#discussion_r317379712
 
 

 ##
 File path: spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala
 ##
 @@ -19,18 +19,22 @@
 
 package org.apache.iceberg.spark
 
+import com.google.common.collect.ImmutableMap
 import com.google.common.collect.Maps
 import java.nio.ByteBuffer
 import java.util
+import java.util.UUID
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{Path, PathFilter}
-import org.apache.iceberg.{DataFile, DataFiles, Metrics, MetricsConfig, 
PartitionSpec}
-import org.apache.iceberg.hadoop.HadoopInputFile
+import org.apache.iceberg._
 
 Review comment:
   I see, will update in next 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] chenjunjiedada commented on a change in pull request #374: Migrate spark table to iceberg table

2019-08-24 Thread GitBox
chenjunjiedada commented on a change in pull request #374: Migrate spark table 
to iceberg table
URL: https://github.com/apache/incubator-iceberg/pull/374#discussion_r317379695
 
 

 ##
 File path: hive/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java
 ##
 @@ -102,7 +102,7 @@ private TServer newThriftServer(TServerSocket socket, 
HiveConf conf) throws Exce
 .transportFactory(new TTransportFactory())
 .protocolFactory(new TBinaryProtocol.Factory())
 .minWorkerThreads(3)
-.maxWorkerThreads(5);
+.maxWorkerThreads(8);
 
 Review comment:
   It is a new unit test I added, not related to the previous.
   
   The default release time is referred to 
`hive.server2.thrift.http.worker.keepalive.time` which is the `stopTimeoutVal` 
in TThreadPoolServer and is set default 60s ( I referred to 
`hive.server2.thrift.http.max.idle.time` by mistake..). 


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 #406: Throw an exception when using Iceberg with Spark embedded metastore

2019-08-24 Thread GitBox
jerryshao commented on issue #406: Throw an exception when using Iceberg with 
Spark embedded metastore
URL: https://github.com/apache/incubator-iceberg/pull/406#issuecomment-524592661
 
 
   > This is only a problem if the embedded metastore DB is open in another 
process or doesn't have the LOCKS table, right?
   
   The problem is that embedded metastore only allows one client at a time, and 
we use client pool for the opened clients, if iceberg opens a client first, 
then Spark itself cannot create a client anymore. Also if Spark opens a client 
first, then iceberg will throw a bunch exceptions.
   
   Typically with such exceptions, iceberg hive catalog cannot be used anymore, 
catch exceptions cannot recover anything, just give users some useful 
information.
   
   I will try to change to catch and log useful information. 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] rdblue opened a new pull request #411: Ignore unsupported partition transforms for forward compatibility

2019-08-24 Thread GitBox
rdblue opened a new pull request #411: Ignore unsupported partition transforms 
for forward compatibility
URL: https://github.com/apache/incubator-iceberg/pull/411
 
 
   


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-24 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_r317377838
 
 

 ##
 File path: spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala
 ##
 @@ -297,5 +301,81 @@ object SparkTableUtil {
   )
 }
   }
+
+  private 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
+  }
+
+  /**
+   * Import a spark table to a iceberg table.
+   *
+   * The import 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 source the database name of the table to be import
+   * @param location the location used to store table metadata
+   *
+   * @return table the imported table
+   */
+  def importSparkTable(source: TableIdentifier, location: String): Table = {
+val sparkSession = SparkSession.builder().getOrCreate()
+import sparkSession.sqlContext.implicits._
+
+val dbName = source.database.getOrElse("default")
+val tableName = source.table
+
+if (!sparkSession.catalog.tableExists(dbName, tableName)) {
+  throw new NoSuchTableException(s"Table $dbName.$tableName does not 
exist")
+}
+
+val partitionSpec = SparkSchemaUtil.specForTable(sparkSession, 
s"$dbName.$tableName")
+val conf = sparkSession.sparkContext.hadoopConfiguration
+val tables = new HadoopTables(conf)
+val schema = SparkSchemaUtil.schemaForTable(sparkSession, 
s"$dbName.$tableName")
+val table = tables.create(schema, partitionSpec, ImmutableMap.of(), 
location)
+val appender = table.newAppend()
+
+if (partitionSpec == PartitionSpec.unpartitioned) {
 
 Review comment:
   Why is there a special case for unpartitioned tables? Doesn't `partitionDF` 
return a single partition in that case?


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 #362: Support create and replace transactions in Catalog

2019-08-24 Thread GitBox
rdblue 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_r317377463
 
 

 ##
 File path: core/src/main/java/org/apache/iceberg/BaseTransaction.java
 ##
 @@ -241,7 +234,14 @@ private void commitCreateTransaction() {
 }
   }
 
-  private void commitReplaceTransaction() {
+  private void commitReplaceTransaction(boolean orCreate) {
+if (base == null && !orCreate) {
+  throw new NoSuchTableException("Table doesn't exist");
+} else if (base == null) {
+  commitCreateTransaction();
 
 Review comment:
   Isn't base going to be reinitialized correctly just before the commit? I 
think the right place to do this is in retry.


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-24 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_r317377271
 
 

 ##
 File path: core/src/main/java/org/apache/iceberg/BaseTransaction.java
 ##
 @@ -241,7 +234,14 @@ private void commitCreateTransaction() {
 }
   }
 
-  private void commitReplaceTransaction() {
+  private void commitReplaceTransaction(boolean orCreate) {
+if (base == null && !orCreate) {
+  throw new NoSuchTableException("Table doesn't exist");
+} else if (base == null) {
+  commitCreateTransaction();
 
 Review comment:
   If `base` is null, I think we can either use `replaceMetadata` instead of 
`base` and keep the existing try-catch or we can actually call 
`commitCreateTransaction` wrapped in another try first. If we get 
`AlreadyExistsException`, we can call the second one, which will replace the 
table.


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] jbapple commented on a change in pull request #348: Avoid floating point number ordering NaN semantics

2019-08-24 Thread GitBox
jbapple commented on a change in pull request #348: Avoid floating point number 
ordering NaN semantics
URL: https://github.com/apache/incubator-iceberg/pull/348#discussion_r317376011
 
 

 ##
 File path: site/docs/spec.md
 ##
 @@ -206,19 +206,22 @@ The schema of a manifest file is a struct called 
`manifest_entry` with the follo
 | **`104  file_size_in_bytes`** | `long`| 
Total file size in bytes

 |
 | ~~**`105 block_size_in_bytes`**~~ | `long`| 
**Deprecated. Always write a default value and do not read.**   

 |
 | **`106  file_ordinal`**   | `optional int`| 
Ordinal of the file w.r.t files with the same partition tuple and snapshot id   

 |
-| **`107  sort_columns`**   | `optional list`   | 
Columns the file is sorted by   

 |
+| **`107  sort_columns`**   | `optional list`   | 
Columns the file is sorted by [2]. If a column has type `float` or `double` and 
contains `NaN`, it must not be in `sort_columns`.   
 |
 
 Review comment:
   Can a column containing nulls be in `sort_columns`? If so, are the nulls at 
the beginning, the end, either, or arbitrarily interspersed?


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] jbapple commented on issue #349: Remove unreferenced hash computations

2019-08-24 Thread GitBox
jbapple commented on issue #349: Remove unreferenced hash computations
URL: https://github.com/apache/incubator-iceberg/pull/349#issuecomment-524583920
 
 
   Moved `boolean` and floating point types to a new table.


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 #348: Avoid floating point number ordering NaN semantics

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #348: Avoid floating point number 
ordering NaN semantics
URL: https://github.com/apache/incubator-iceberg/pull/348#discussion_r317375385
 
 

 ##
 File path: site/docs/spec.md
 ##
 @@ -206,19 +206,21 @@ The schema of a manifest file is a struct called 
`manifest_entry` with the follo
 | **`104  file_size_in_bytes`** | `long`| 
Total file size in bytes

 |
 | ~~**`105 block_size_in_bytes`**~~ | `long`| 
**Deprecated. Always write a default value and do not read.**   

 |
 | **`106  file_ordinal`**   | `optional int`| 
Ordinal of the file w.r.t files with the same partition tuple and snapshot id   

 |
-| **`107  sort_columns`**   | `optional list`   | 
Columns the file is sorted by   

 |
+| **`107  sort_columns`**   | `optional list`   | 
Columns the file is sorted by [2]. If a column has type `float` or `double` and 
contains `NaN`, it must not be in `sort_columns`.   
 |
 | **`108  column_sizes`**   | `optional map`| 
Map from column id to the total size on disk of all regions that store the 
column. Does not include bytes necessary to read other columns, like footers. 
Leave null for row-oriented formats (Avro). |
 | **`109  value_counts`**   | `optional map`| 
Map from column id to number of values in the column (including null values)

 |
 | **`110  null_value_counts`**  | `optional map`| 
Map from column id to number of null values in the column   

 |
 | ~~**`111 distinct_counts`**~~ | `optional map`| 
**Deprecated. Do not use.** 

 |
-| **`125  lower_bounds`**   | `optional map<126: int, 127: binary>` | 
Map from column id to lower bound in the column serialized as binary [1]. Each 
value must be less than or equal to all values in the column for the file.  
  |
-| **`128  upper_bounds`**   | `optional map<129: int, 130: binary>` | 
Map from column id to upper bound in the column serialized as binary [1]. Each 
value must be greater than or equal to all values in the column for the file.   
  |
+| **`125  lower_bounds`**   | `optional map<126: int, 127: binary>` | 
Map from column id to lower bound in the column serialized as binary [1]. Each 
value must be less than or equal to all values in the column for the file. [3]  
  |
+| **`128  upper_bounds`**   | `optional map<129: int, 130: binary>` | 
Map from column id to upper bound in the column serialized as binary [1]. Each 
value must be greater than or equal to all values in the column for the file. 
[3] |
 | **`131  key_metadata`**   | `optional binary` | 
Implementation-specific key metadata for encryption 

 |
 | **`132  split_offsets`**  | `optional list`   | 
Split offsets for the data file. For example, all row group offsets in a 
Parquet file. Must be sorted ascending. 
|
 
 Notes:
 
 1. Single-value serialization for lower and upper bounds is detailed in 
Appendix D.
+2. For `float` and `double`, the value `-0.0` must precede `+0.0`, as in the 
IEEE 754 `totalOrder` predicate.
+3. Since `NaN` is not less than or equal to or greater than any value, this 
implies that columns of type `float` or `double` may not appear in 
`lower_bounds` or `upper_bounds` when the column contains `NaN`. As for `float` 

[GitHub] [incubator-iceberg] rdblue commented on issue #349: Remove unreferenced hash computations

2019-08-24 Thread GitBox
rdblue commented on issue #349: Remove unreferenced hash computations
URL: https://github.com/apache/incubator-iceberg/pull/349#issuecomment-524583239
 
 
   > I'm confused. They are valid types for bucketing?
   
   No, I just left out the "not" by accident.


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 #315: [WIP] Incremental processing prototype

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #315: [WIP] Incremental processing 
prototype
URL: https://github.com/apache/incubator-iceberg/pull/315#discussion_r317375296
 
 

 ##
 File path: api/src/main/java/org/apache/iceberg/Table.java
 ##
 @@ -44,6 +44,15 @@
*/
   TableScan newScan();
 
+
+  /**
+   * @param fromSnapshotId - the last snapshot id read by the user, exclusive
+   * @param toSnapshotId - read incremental data upto this snapshot id
+   * @return a table scan which can read incremental data from {@param 
fromSnapshotId}
+   * exclusive and up to {@toSnapshotId} inclusive
+   */
+  TableScan newIncrementalScan(long fromSnapshotId, long toSnapshotId);
 
 Review comment:
   You may also consider adding `newIncrementalScan(long fromSnapshotId)` that 
uses `currentSnapshot` for `toSnapshotId`. We would need a method to get the 
scan's "to" snapshot, though, so that it can be saved as reader state.


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 edited a comment on issue #349: Remove unreferenced hash computations

2019-08-24 Thread GitBox
rdblue edited a comment on issue #349: Remove unreferenced hash computations
URL: https://github.com/apache/incubator-iceberg/pull/349#issuecomment-524581190
 
 
   I'm reluctant to completely remove these in case there is some legitimate 
reason to hash values in the future that requires specification. I'm concerned 
if we did this a second time, I would forget that the hash values for 
equivalent floats and doubles are the same to support type promotion.
   
   Instead of removing these entirely, what about moving them to a separate 
table with a note? "Boolean, float, and double values are NOT valid types for 
bucketing, but in the event that the values should be hashed, the above hash 
function must be applied as in the following table."


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 #315: [WIP] Incremental processing prototype

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #315: [WIP] Incremental processing 
prototype
URL: https://github.com/apache/incubator-iceberg/pull/315#discussion_r317375264
 
 

 ##
 File path: api/src/main/java/org/apache/iceberg/Table.java
 ##
 @@ -44,6 +44,15 @@
*/
   TableScan newScan();
 
+
+  /**
+   * @param fromSnapshotId - the last snapshot id read by the user, exclusive
 
 Review comment:
   Nit: this needs a description in addition to params.


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 #315: [WIP] Incremental processing prototype

2019-08-24 Thread GitBox
rdblue commented on issue #315: [WIP] Incremental processing prototype
URL: https://github.com/apache/incubator-iceberg/pull/315#issuecomment-524582998
 
 
   @rdsr, thanks for working on this! I really like how clean this version is. 
Definitely improving quickly!


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 #315: [WIP] Incremental processing prototype

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #315: [WIP] Incremental processing 
prototype
URL: https://github.com/apache/incubator-iceberg/pull/315#discussion_r317375221
 
 

 ##
 File path: core/src/test/java/org/apache/iceberg/TestIncrementalScan.java
 ##
 @@ -0,0 +1,136 @@
+/*
+ * 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;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.util.List;
+import java.util.Set;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestIncrementalScan extends TableTestBase {
 
 Review comment:
   I think this should probably test the cases I mentioned above: rollback and 
2-append transactions.


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 #315: [WIP] Incremental processing prototype

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #315: [WIP] Incremental processing 
prototype
URL: https://github.com/apache/incubator-iceberg/pull/315#discussion_r317375217
 
 

 ##
 File path: core/src/main/java/org/apache/iceberg/IncrementalDataScan.java
 ##
 @@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Sets;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+
+class IncrementalDataScan extends DataTableScan implements TableScan {
+  private final TableOperations ops;
+  private final Table table;
+  private boolean colStats;
+  private long fromSnapshotId;
+  private long toSnapshotId;
+  private Set snapshotIds;
+
+
+  IncrementalDataScan(TableOperations ops, Table table, long fromSnapshotId, 
long toSnapshotId) {
+super(ops, table);
+this.ops = ops;
+this.table = table;
+this.fromSnapshotId = fromSnapshotId;
+this.toSnapshotId = toSnapshotId;
+this.snapshotIds = snapshotsWithin(table, fromSnapshotId, toSnapshotId);
+  }
+
+  /**
+   * @return all snapshots ids between {@param fromSnapshotId } exclusive
+   * and {@param toSnapshotId} inclusive
+   */
+  private static Set snapshotsWithin(Table table, long fromSnapshotId, 
long toSnapshotId) {
+Set snapshotIds = Sets.newHashSet();
+// table history is ordered
+Iterator historyEntryIterator = table.history().iterator();
+while (historyEntryIterator.hasNext()) {
+  if (historyEntryIterator.next().snapshotId() == fromSnapshotId) {
+break;
+  }
+}
+while (historyEntryIterator.hasNext()) {
+  long snapshotId = historyEntryIterator.next().snapshotId();
+  snapshotIds.add(snapshotId);
+  if (snapshotId == toSnapshotId) {
+break;
+  }
+}
+return snapshotIds;
+  }
+
+  private IncrementalDataScan(TableOperations ops, Table table, Long 
snapshotId, Schema schema,
+  Expression rowFilter, boolean caseSensitive, 
boolean colStats,
+  Collection selectedColumns,
+  Long fromSnapshotId, Long toSnapshotId) {
+super(ops, table, snapshotId, schema, rowFilter, caseSensitive, colStats, 
selectedColumns);
+this.ops = ops;
+this.table = table;
+this.colStats = colStats;
+this.fromSnapshotId = fromSnapshotId;
+this.toSnapshotId = toSnapshotId;
+  }
+
+
+  @Override
+  public CloseableIterable planFiles() {
+Snapshot snapshot = table.snapshot(toSnapshotId);
+Preconditions.checkState(snapshot != null, "Cannot find snapshot 
corresponding to snapshot id: %s", toSnapshotId);
+//TODO publish an incremental scan event
+return planFiles(ops, snapshot, filter(), isCaseSensitive(), colStats);
+  }
+
+  @Override
+  @SuppressWarnings("checkstyle:HiddenField")
+  public CloseableIterable planFiles(
+  TableOperations ops, Snapshot snapshot, Expression rowFilter, 
boolean caseSensitive, boolean colStats) {
 
 Review comment:
   I think it might be a bit cleaner to implement `planFiles` using 
`Snapshot.addedFiles` since this has to add methods to pass in predicates for 
matching manifests and manifest entries. The advantage of `addedFiles` is that 
you don't need to scan compacted manifests from replace commits. Ignoring those 
large manifests is probably faster in most cases, especially if your writer is 
using `FastAppend`.


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 #315: [WIP] Incremental processing prototype

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #315: [WIP] Incremental processing 
prototype
URL: https://github.com/apache/incubator-iceberg/pull/315#discussion_r317375092
 
 

 ##
 File path: core/src/main/java/org/apache/iceberg/IncrementalDataScan.java
 ##
 @@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Sets;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+
+class IncrementalDataScan extends DataTableScan implements TableScan {
+  private final TableOperations ops;
+  private final Table table;
+  private boolean colStats;
+  private long fromSnapshotId;
+  private long toSnapshotId;
+  private Set snapshotIds;
+
+
+  IncrementalDataScan(TableOperations ops, Table table, long fromSnapshotId, 
long toSnapshotId) {
+super(ops, table);
+this.ops = ops;
+this.table = table;
+this.fromSnapshotId = fromSnapshotId;
+this.toSnapshotId = toSnapshotId;
+this.snapshotIds = snapshotsWithin(table, fromSnapshotId, toSnapshotId);
+  }
+
+  /**
+   * @return all snapshots ids between {@param fromSnapshotId } exclusive
+   * and {@param toSnapshotId} inclusive
+   */
+  private static Set snapshotsWithin(Table table, long fromSnapshotId, 
long toSnapshotId) {
 
 Review comment:
   I don't think it is correct to use table history for this. Table history is 
a log of changes to the current snapshot ID.
   
   When a table is rolled back, the current snapshot ID is set explicitly. So 
the snapshot log might go from snapshot A, to B, to C, back to B, and then to 
D. Using the history table for the snapshot range A to D would result in 
processing C, even though it was rolled back, and processing B twice.
   
   Also, the snapshot log can be missing entries when transactions are used. 
Transactions are are implemented by staging changes without committing, and 
then committing the final result and the latest table state. If you had a 
transaction of two append operations based on snapshot A, the table would 
contain snapshots A, B, and C, where the parent of C is B and the parent of B 
is A. But the history log will only show that the current snapshot went from A 
to C.
   
   I think a better way to get changes is to use the parent relationship 
between snapshots. That way, if you have history A, to C, to D, to C, you'd get 
C, B, and A. 
[`SnapshotUtil`](https://github.com/apache/incubator-iceberg/blob/master/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java#L28-L58)
 has some code that traverses ancestors.
   
   Also, it is critical that you skip `replace` snapshots because the data has 
not changed, but grouping in files has. If you only process added files and 
skip replace then you'll not process data twice. Also, delete and overwrite 
should probably throw an exception?


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] jbapple commented on issue #349: Remove unreferenced hash computations

2019-08-24 Thread GitBox
jbapple commented on issue #349: Remove unreferenced hash computations
URL: https://github.com/apache/incubator-iceberg/pull/349#issuecomment-524581599
 
 
   > Boolean, float, and double values are valid types for bucketing, but in 
the event that the values should be hashed, the above hash function must be 
applied as in the following table.
   
   I'm confused. They *are* valid types for bucketing?
   
   


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 #315: [WIP] Incremental processing prototype

2019-08-24 Thread GitBox
rdblue commented on issue #315: [WIP] Incremental processing prototype
URL: https://github.com/apache/incubator-iceberg/pull/315#issuecomment-524581490
 
 
   @rdsr, sorry for the delay! I didn't notice that this was updated or see 
your request. Sorry!


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] jbapple commented on a change in pull request #348: Avoid floating point number ordering NaN semantics

2019-08-24 Thread GitBox
jbapple commented on a change in pull request #348: Avoid floating point number 
ordering NaN semantics
URL: https://github.com/apache/incubator-iceberg/pull/348#discussion_r317374691
 
 

 ##
 File path: site/docs/spec.md
 ##
 @@ -206,19 +206,21 @@ The schema of a manifest file is a struct called 
`manifest_entry` with the follo
 | **`104  file_size_in_bytes`** | `long`| 
Total file size in bytes

 |
 | ~~**`105 block_size_in_bytes`**~~ | `long`| 
**Deprecated. Always write a default value and do not read.**   

 |
 | **`106  file_ordinal`**   | `optional int`| 
Ordinal of the file w.r.t files with the same partition tuple and snapshot id   

 |
-| **`107  sort_columns`**   | `optional list`   | 
Columns the file is sorted by   

 |
+| **`107  sort_columns`**   | `optional list`   | 
Columns the file is sorted by [2]. If a column has type `float` or `double` and 
contains `NaN`, it must not be in `sort_columns`.   
 |
 | **`108  column_sizes`**   | `optional map`| 
Map from column id to the total size on disk of all regions that store the 
column. Does not include bytes necessary to read other columns, like footers. 
Leave null for row-oriented formats (Avro). |
 | **`109  value_counts`**   | `optional map`| 
Map from column id to number of values in the column (including null values)

 |
 | **`110  null_value_counts`**  | `optional map`| 
Map from column id to number of null values in the column   

 |
 | ~~**`111 distinct_counts`**~~ | `optional map`| 
**Deprecated. Do not use.** 

 |
-| **`125  lower_bounds`**   | `optional map<126: int, 127: binary>` | 
Map from column id to lower bound in the column serialized as binary [1]. Each 
value must be less than or equal to all values in the column for the file.  
  |
-| **`128  upper_bounds`**   | `optional map<129: int, 130: binary>` | 
Map from column id to upper bound in the column serialized as binary [1]. Each 
value must be greater than or equal to all values in the column for the file.   
  |
+| **`125  lower_bounds`**   | `optional map<126: int, 127: binary>` | 
Map from column id to lower bound in the column serialized as binary [1]. Each 
value must be less than or equal to all values in the column for the file. [3]  
  |
+| **`128  upper_bounds`**   | `optional map<129: int, 130: binary>` | 
Map from column id to upper bound in the column serialized as binary [1]. Each 
value must be greater than or equal to all values in the column for the file. 
[3] |
 | **`131  key_metadata`**   | `optional binary` | 
Implementation-specific key metadata for encryption 

 |
 | **`132  split_offsets`**  | `optional list`   | 
Split offsets for the data file. For example, all row group offsets in a 
Parquet file. Must be sorted ascending. 
|
 
 Notes:
 
 1. Single-value serialization for lower and upper bounds is detailed in 
Appendix D.
+2. For `float` and `double`, the value `-0.0` must precede `+0.0`, as in the 
IEEE 754 `totalOrder` predicate.
+3. Since `NaN` is not less than or equal to or greater than any value, this 
implies that columns of type `float` or `double` may not appear in 
`lower_bounds` or `upper_bounds` when the column contains `NaN`. As for `float` 

[GitHub] [incubator-iceberg] rdblue commented on issue #349: Remove unreferenced hash computations

2019-08-24 Thread GitBox
rdblue commented on issue #349: Remove unreferenced hash computations
URL: https://github.com/apache/incubator-iceberg/pull/349#issuecomment-524581190
 
 
   I'm reluctant to completely remove these in case there is some legitimate 
reason to hash values in the future that requires specification. I'm concerned 
if we did this a second time, I would forget that the hash values for 
equivalent floats and doubles are the same to support type promotion.
   
   Instead of removing these entirely, what about moving them to a separate 
table with a note? "Boolean, float, and double values are valid types for 
bucketing, but in the event that the values should be hashed, the above hash 
function must be applied as in the following table."


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 #348: Avoid floating point number ordering NaN semantics

2019-08-24 Thread GitBox
rdblue commented on issue #348: Avoid floating point number ordering NaN 
semantics
URL: https://github.com/apache/incubator-iceberg/pull/348#issuecomment-524580593
 
 
   Thanks for looking into this, @jbapple! Sorry for the delay reviewing 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] rdblue commented on a change in pull request #348: Avoid floating point number ordering NaN semantics

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #348: Avoid floating point number 
ordering NaN semantics
URL: https://github.com/apache/incubator-iceberg/pull/348#discussion_r317374413
 
 

 ##
 File path: site/docs/spec.md
 ##
 @@ -206,19 +206,21 @@ The schema of a manifest file is a struct called 
`manifest_entry` with the follo
 | **`104  file_size_in_bytes`** | `long`| 
Total file size in bytes

 |
 | ~~**`105 block_size_in_bytes`**~~ | `long`| 
**Deprecated. Always write a default value and do not read.**   

 |
 | **`106  file_ordinal`**   | `optional int`| 
Ordinal of the file w.r.t files with the same partition tuple and snapshot id   

 |
-| **`107  sort_columns`**   | `optional list`   | 
Columns the file is sorted by   

 |
+| **`107  sort_columns`**   | `optional list`   | 
Columns the file is sorted by [2]. If a column has type `float` or `double` and 
contains `NaN`, it must not be in `sort_columns`.   
 |
 | **`108  column_sizes`**   | `optional map`| 
Map from column id to the total size on disk of all regions that store the 
column. Does not include bytes necessary to read other columns, like footers. 
Leave null for row-oriented formats (Avro). |
 | **`109  value_counts`**   | `optional map`| 
Map from column id to number of values in the column (including null values)

 |
 | **`110  null_value_counts`**  | `optional map`| 
Map from column id to number of null values in the column   

 |
 | ~~**`111 distinct_counts`**~~ | `optional map`| 
**Deprecated. Do not use.** 

 |
-| **`125  lower_bounds`**   | `optional map<126: int, 127: binary>` | 
Map from column id to lower bound in the column serialized as binary [1]. Each 
value must be less than or equal to all values in the column for the file.  
  |
-| **`128  upper_bounds`**   | `optional map<129: int, 130: binary>` | 
Map from column id to upper bound in the column serialized as binary [1]. Each 
value must be greater than or equal to all values in the column for the file.   
  |
+| **`125  lower_bounds`**   | `optional map<126: int, 127: binary>` | 
Map from column id to lower bound in the column serialized as binary [1]. Each 
value must be less than or equal to all values in the column for the file. [3]  
  |
+| **`128  upper_bounds`**   | `optional map<129: int, 130: binary>` | 
Map from column id to upper bound in the column serialized as binary [1]. Each 
value must be greater than or equal to all values in the column for the file. 
[3] |
 | **`131  key_metadata`**   | `optional binary` | 
Implementation-specific key metadata for encryption 

 |
 | **`132  split_offsets`**  | `optional list`   | 
Split offsets for the data file. For example, all row group offsets in a 
Parquet file. Must be sorted ascending. 
|
 
 Notes:
 
 1. Single-value serialization for lower and upper bounds is detailed in 
Appendix D.
+2. For `float` and `double`, the value `-0.0` must precede `+0.0`, as in the 
IEEE 754 `totalOrder` predicate.
+3. Since `NaN` is not less than or equal to or greater than any value, this 
implies that columns of type `float` or `double` may not appear in 
`lower_bounds` or `upper_bounds` when the column contains `NaN`. As for `float` 

[GitHub] [incubator-iceberg] aokolnychyi opened a new issue #410: Refactor AssertHelpers and TestHelpers

2019-08-24 Thread GitBox
aokolnychyi opened a new issue #410: Refactor AssertHelpers and TestHelpers
URL: https://github.com/apache/incubator-iceberg/issues/410
 
 
   We should refactor `TestHelpers` in iceberg-api and `AssertHelpers` in 
iceberg-core as they share a lot of code.


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 #280: Add persistent IDs to partition fields

2019-08-24 Thread GitBox
rdblue commented on issue #280: Add persistent IDs to partition fields
URL: 
https://github.com/apache/incubator-iceberg/issues/280#issuecomment-524579846
 
 
   @manishmalhotrawork, we need to keep track of IDs that have been assigned to 
partition fields in a table and reuse them when partition specs change. They 
should probably continue to start at 1,000.
   
   @timmylicheng, Schema field IDs are the integers passed in when creating 
struct fields, maps, and lists. See http://iceberg.apache.org/api/#nested-types


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-24 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-524579905
 
 
   I merged #387 so this is unblocked.


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 #362: Support create and replace transactions in Catalog

2019-08-24 Thread GitBox
rdblue 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_r317374086
 
 

 ##
 File path: 
hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
 ##
 @@ -0,0 +1,238 @@
+/*
+ * 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 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.Assert;
+import org.junit.Before;
+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 extends HiveMetastoreTest {
+
+  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();
+
+  @Rule
+  public ExpectedException exceptionRule = ExpectedException.none();
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation;
+
+  @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);
+exceptionRule.expectMessage("Metastore operation failed");
+
+Assert.assertFalse("Table should not exist", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+Transaction txn = catalog.newCreateTableTransaction(
+TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap());
+
+// create the table concurrently
+catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC);
+Assert.assertTrue("Table should be created", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+// expect the transaction to fail
+txn.commitTransaction();
 
 Review comment:
   `AssertHelpers` is the one in core, so probably use that one.
   
   Before we were producing test Jars to share code, we copied this in a few 
places. Should probably clean that up.


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-24 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_r317373956
 
 

 ##
 File path: spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala
 ##
 @@ -297,5 +301,81 @@ object SparkTableUtil {
   )
 }
   }
+
+  private 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) {
 
 Review comment:
   Style: please use `foreach` instead of `for` loops in scala.


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-24 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_r317373945
 
 

 ##
 File path: spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala
 ##
 @@ -297,5 +301,81 @@ object SparkTableUtil {
   )
 }
   }
+
+  private def buildManifest(table: Table,
+sparkDataFiles: Seq[SparkDataFile],
 
 Review comment:
   Nit: indentation is off.


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-24 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_r317373916
 
 

 ##
 File path: hive/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java
 ##
 @@ -102,7 +102,7 @@ private TServer newThriftServer(TServerSocket socket, 
HiveConf conf) throws Exce
 .transportFactory(new TTransportFactory())
 .protocolFactory(new TBinaryProtocol.Factory())
 .minWorkerThreads(3)
-.maxWorkerThreads(5);
+.maxWorkerThreads(8);
 
 Review comment:
   What have previous tests not released? What is the "default release time"?


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-24 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_r317373882
 
 

 ##
 File path: hive/src/test/java/org/apache/iceberg/hive/TestHiveMetastore.java
 ##
 @@ -111,6 +111,7 @@ private HiveConf newHiveConf(int port) {
 HiveConf newHiveConf = new HiveConf(new Configuration(), 
TestHiveMetastore.class);
 newHiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, 
"thrift://localhost:" + port);
 newHiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, "file:" + 
hiveLocalDir.getAbsolutePath());
+newHiveConf.set(HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname, 
"false");
 
 Review comment:
   What warning?


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-24 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_r317373878
 
 

 ##
 File path: spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala
 ##
 @@ -19,18 +19,22 @@
 
 package org.apache.iceberg.spark
 
+import com.google.common.collect.ImmutableMap
 import com.google.common.collect.Maps
 import java.nio.ByteBuffer
 import java.util
+import java.util.UUID
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{Path, PathFilter}
-import org.apache.iceberg.{DataFile, DataFiles, Metrics, MetricsConfig, 
PartitionSpec}
-import org.apache.iceberg.hadoop.HadoopInputFile
+import org.apache.iceberg._
 
 Review comment:
   We never use wildcard imports.


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 closed issue #366: Concurrent appends to the same table using HiveCatalog lead to deadlock in the Metastore

2019-08-24 Thread GitBox
rdblue closed issue #366: Concurrent appends to the same table using 
HiveCatalog lead to deadlock in the Metastore
URL: https://github.com/apache/incubator-iceberg/issues/366
 
 
   


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 #401: Introduce a wait before each check lock state request to HMS

2019-08-24 Thread GitBox
rdblue merged pull request #401: Introduce a wait before each check lock state 
request to HMS
URL: https://github.com/apache/incubator-iceberg/pull/401
 
 
   


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 #401: Introduce a wait before each check lock state request to HMS

2019-08-24 Thread GitBox
rdblue commented on issue #401: Introduce a wait before each check lock state 
request to HMS
URL: https://github.com/apache/incubator-iceberg/pull/401#issuecomment-524579110
 
 
   +1. I'll merge this. Thanks for fixing it, @aokolnychyi!


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 #357: Add in and not in predicates

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #357: Add in and not in predicates
URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r317373703
 
 

 ##
 File path: api/src/test/java/org/apache/iceberg/expressions/TestEvaluator.java
 ##
 @@ -362,4 +368,197 @@ public void testCharSeqValue() {
 Assert.assertFalse("string(abc) == utf8(abcd) => false",
 evaluator.eval(TestHelpers.Row.of(new Utf8("abcd";
   }
+
+  @Test
+  public void testIn() {
+Assert.assertEquals(3, in("s", 7, 8, 9).literals().size());
+Assert.assertEquals(3, in("s", 7, 8.1, Long.MAX_VALUE).literals().size());
+Assert.assertEquals(2, in("s", "abc", "abd", "abc").literals().size());
+Assert.assertEquals(0, in("s").literals().size());
+Assert.assertEquals(1, in("s", 5).literals().size());
+Assert.assertEquals(1, in("s", 5, 5).literals().size());
+Assert.assertEquals(1, in("s", Arrays.asList(5, 5)).literals().size());
+Assert.assertEquals(0, in("s", Collections.emptyList()).literals().size());
+
+Evaluator evaluator = new Evaluator(STRUCT, in("x", 7, 8, Long.MAX_VALUE));
+Assert.assertTrue("7 in [7, 8] => true", 
evaluator.eval(TestHelpers.Row.of(7, 8, null)));
+Assert.assertFalse("9 in [7, 8]  => false", 
evaluator.eval(TestHelpers.Row.of(9, 8, null)));
+
+Evaluator longEvaluator = new Evaluator(STRUCT,
+in("x", Long.MAX_VALUE, Integer.MAX_VALUE, Long.MIN_VALUE));
+Assert.assertTrue("Integer.MAX_VALUE in [Integer.MAX_VALUE] => true",
+longEvaluator.eval(TestHelpers.Row.of(Integer.MAX_VALUE, 7.0, null)));
+Assert.assertFalse("6 in [Integer.MAX_VALUE]  => false",
+longEvaluator.eval(TestHelpers.Row.of(6, 6.8, null)));
+
+Evaluator integerEvaluator = new Evaluator(STRUCT, in("y", 7, 8, 9.1));
+Assert.assertTrue("7.0 in [7, 8, 9.1] => true", 
integerEvaluator.eval(TestHelpers.Row.of(7, 7.0, null)));
+Assert.assertTrue("9.1 in [7, 8, 9.1] => true", 
integerEvaluator.eval(TestHelpers.Row.of(7, 9.1, null)));
+Assert.assertFalse("6.8 in [7, 8, 9]  => false", 
integerEvaluator.eval(TestHelpers.Row.of(6, 6.8, null)));
+
+Evaluator structEvaluator = new Evaluator(STRUCT, in("s1.s2.s3.s4.i", 7, 
8, 9));
+Assert.assertTrue("7 in [7, 8, 9] => true",
+structEvaluator.eval(TestHelpers.Row.of(7, 8, null,
+TestHelpers.Row.of(
+TestHelpers.Row.of(
+TestHelpers.Row.of(
+TestHelpers.Row.of(7)));
+Assert.assertFalse("6 in [7, 8, 9]  => false",
+structEvaluator.eval(TestHelpers.Row.of(6, 8, null,
+TestHelpers.Row.of(
+TestHelpers.Row.of(
+TestHelpers.Row.of(
+TestHelpers.Row.of(6)));
+
+StructType charSeqStruct = StructType.of(required(34, "s", 
Types.StringType.get()));
+Evaluator charSeqEvaluator = new Evaluator(charSeqStruct, in("s", "abc", 
"abd", "abc"));
+Assert.assertTrue("utf8(abc) in [string(abc), string(abd)] => true",
+charSeqEvaluator.eval(TestHelpers.Row.of(new Utf8("abc";
+Assert.assertFalse("utf8(abcd) in [string(abc), string(abd)] => false",
+charSeqEvaluator.eval(TestHelpers.Row.of(new Utf8("abcd";
+  }
+
+  @Test
+  public void testInExceptions() {
+TestHelpers.assertThrows(
+"Throw exception if value is null",
+NullPointerException.class,
+"Cannot create expression literal from null",
+() -> in("x", (Literal) null));
+
+TestHelpers.assertThrows(
+"Throw exception if value is null",
+NullPointerException.class,
+"Values cannot be null for IN predicate",
+() -> in("x", (Collection) null));
+
+TestHelpers.assertThrows(
+"Throw exception if calling literals() for EQ predicate",
+IllegalArgumentException.class,
+"EQ predicate cannot return a list of literals",
+() -> equal("x", 5).literals());
 
 Review comment:
   I don't think this is necessary. This should return `ImmutableList.of(5)`.


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 #357: Add in and not in predicates

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #357: Add in and not in predicates
URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r317373662
 
 

 ##
 File path: api/src/test/java/org/apache/iceberg/expressions/TestEvaluator.java
 ##
 @@ -362,4 +368,197 @@ public void testCharSeqValue() {
 Assert.assertFalse("string(abc) == utf8(abcd) => false",
 evaluator.eval(TestHelpers.Row.of(new Utf8("abcd";
   }
+
+  @Test
+  public void testIn() {
+Assert.assertEquals(3, in("s", 7, 8, 9).literals().size());
+Assert.assertEquals(3, in("s", 7, 8.1, Long.MAX_VALUE).literals().size());
+Assert.assertEquals(2, in("s", "abc", "abd", "abc").literals().size());
+Assert.assertEquals(0, in("s").literals().size());
+Assert.assertEquals(1, in("s", 5).literals().size());
+Assert.assertEquals(1, in("s", 5, 5).literals().size());
+Assert.assertEquals(1, in("s", Arrays.asList(5, 5)).literals().size());
+Assert.assertEquals(0, in("s", Collections.emptyList()).literals().size());
+
+Evaluator evaluator = new Evaluator(STRUCT, in("x", 7, 8, Long.MAX_VALUE));
+Assert.assertTrue("7 in [7, 8] => true", 
evaluator.eval(TestHelpers.Row.of(7, 8, null)));
+Assert.assertFalse("9 in [7, 8]  => false", 
evaluator.eval(TestHelpers.Row.of(9, 8, null)));
+
+Evaluator longEvaluator = new Evaluator(STRUCT,
+in("x", Long.MAX_VALUE, Integer.MAX_VALUE, Long.MIN_VALUE));
+Assert.assertTrue("Integer.MAX_VALUE in [Integer.MAX_VALUE] => true",
+longEvaluator.eval(TestHelpers.Row.of(Integer.MAX_VALUE, 7.0, null)));
+Assert.assertFalse("6 in [Integer.MAX_VALUE]  => false",
+longEvaluator.eval(TestHelpers.Row.of(6, 6.8, null)));
+
+Evaluator integerEvaluator = new Evaluator(STRUCT, in("y", 7, 8, 9.1));
+Assert.assertTrue("7.0 in [7, 8, 9.1] => true", 
integerEvaluator.eval(TestHelpers.Row.of(7, 7.0, null)));
 
 Review comment:
   It is confusing to have `x=7` here because 7.0 (`y`) is the value you're 
testing in the set. Can you change these to `0`?


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 #357: Add in and not in predicates

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #357: Add in and not in predicates
URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r317373638
 
 

 ##
 File path: api/src/test/java/org/apache/iceberg/expressions/TestEvaluator.java
 ##
 @@ -362,4 +368,197 @@ public void testCharSeqValue() {
 Assert.assertFalse("string(abc) == utf8(abcd) => false",
 evaluator.eval(TestHelpers.Row.of(new Utf8("abcd";
   }
+
+  @Test
+  public void testIn() {
+Assert.assertEquals(3, in("s", 7, 8, 9).literals().size());
+Assert.assertEquals(3, in("s", 7, 8.1, Long.MAX_VALUE).literals().size());
+Assert.assertEquals(2, in("s", "abc", "abd", "abc").literals().size());
+Assert.assertEquals(0, in("s").literals().size());
+Assert.assertEquals(1, in("s", 5).literals().size());
+Assert.assertEquals(1, in("s", 5, 5).literals().size());
+Assert.assertEquals(1, in("s", Arrays.asList(5, 5)).literals().size());
+Assert.assertEquals(0, in("s", Collections.emptyList()).literals().size());
+
+Evaluator evaluator = new Evaluator(STRUCT, in("x", 7, 8, Long.MAX_VALUE));
+Assert.assertTrue("7 in [7, 8] => true", 
evaluator.eval(TestHelpers.Row.of(7, 8, null)));
+Assert.assertFalse("9 in [7, 8]  => false", 
evaluator.eval(TestHelpers.Row.of(9, 8, null)));
+
+Evaluator longEvaluator = new Evaluator(STRUCT,
+in("x", Long.MAX_VALUE, Integer.MAX_VALUE, Long.MIN_VALUE));
+Assert.assertTrue("Integer.MAX_VALUE in [Integer.MAX_VALUE] => true",
+longEvaluator.eval(TestHelpers.Row.of(Integer.MAX_VALUE, 7.0, null)));
+Assert.assertFalse("6 in [Integer.MAX_VALUE]  => false",
+longEvaluator.eval(TestHelpers.Row.of(6, 6.8, null)));
+
+Evaluator integerEvaluator = new Evaluator(STRUCT, in("y", 7, 8, 9.1));
+Assert.assertTrue("7.0 in [7, 8, 9.1] => true", 
integerEvaluator.eval(TestHelpers.Row.of(7, 7.0, null)));
+Assert.assertTrue("9.1 in [7, 8, 9.1] => true", 
integerEvaluator.eval(TestHelpers.Row.of(7, 9.1, null)));
+Assert.assertFalse("6.8 in [7, 8, 9]  => false", 
integerEvaluator.eval(TestHelpers.Row.of(6, 6.8, null)));
 
 Review comment:
   This should also be `[7, 8, 9.1]` because the predicate hasn't changed.


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 #357: Add in and not in predicates

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #357: Add in and not in predicates
URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r317373623
 
 

 ##
 File path: api/src/test/java/org/apache/iceberg/expressions/TestEvaluator.java
 ##
 @@ -362,4 +368,197 @@ public void testCharSeqValue() {
 Assert.assertFalse("string(abc) == utf8(abcd) => false",
 evaluator.eval(TestHelpers.Row.of(new Utf8("abcd";
   }
+
+  @Test
+  public void testIn() {
+Assert.assertEquals(3, in("s", 7, 8, 9).literals().size());
+Assert.assertEquals(3, in("s", 7, 8.1, Long.MAX_VALUE).literals().size());
+Assert.assertEquals(2, in("s", "abc", "abd", "abc").literals().size());
+Assert.assertEquals(0, in("s").literals().size());
+Assert.assertEquals(1, in("s", 5).literals().size());
+Assert.assertEquals(1, in("s", 5, 5).literals().size());
+Assert.assertEquals(1, in("s", Arrays.asList(5, 5)).literals().size());
+Assert.assertEquals(0, in("s", Collections.emptyList()).literals().size());
+
+Evaluator evaluator = new Evaluator(STRUCT, in("x", 7, 8, Long.MAX_VALUE));
+Assert.assertTrue("7 in [7, 8] => true", 
evaluator.eval(TestHelpers.Row.of(7, 8, null)));
+Assert.assertFalse("9 in [7, 8]  => false", 
evaluator.eval(TestHelpers.Row.of(9, 8, null)));
+
+Evaluator longEvaluator = new Evaluator(STRUCT,
 
 Review comment:
   This name is misleading because the predicate is on an integer field in the 
schema. Can you rename it to `intSetEvaluator` or something more clear?


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-24 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_r317373320
 
 

 ##
 File path: 
hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
 ##
 @@ -0,0 +1,238 @@
+/*
+ * 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 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.Assert;
+import org.junit.Before;
+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 extends HiveMetastoreTest {
+
+  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();
+
+  @Rule
+  public ExpectedException exceptionRule = ExpectedException.none();
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation;
+
+  @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);
+exceptionRule.expectMessage("Metastore operation failed");
+
+Assert.assertFalse("Table should not exist", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+Transaction txn = catalog.newCreateTableTransaction(
+TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap());
+
+// create the table concurrently
+catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC);
+Assert.assertTrue("Table should be created", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+// expect the transaction to fail
+txn.commitTransaction();
 
 Review comment:
   Wait, what is the difference between `AssertHelpers` and `TestHelpers`?


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 #357: Add in and not in predicates

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #357: Add in and not in predicates
URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r317373269
 
 

 ##
 File path: 
parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java
 ##
 @@ -19,19 +19,14 @@
 
 package org.apache.iceberg.parquet;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Maps;
 import java.util.Map;
+import java.util.Set;
 import java.util.function.Function;
 import org.apache.iceberg.Schema;
-import org.apache.iceberg.expressions.Binder;
-import org.apache.iceberg.expressions.BoundReference;
-import org.apache.iceberg.expressions.Expression;
-import org.apache.iceberg.expressions.ExpressionVisitors;
+import org.apache.iceberg.expressions.*;
 
 Review comment:
   Please clean up the imports in this file and remove the wildcard import.


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-24 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_r317373206
 
 

 ##
 File path: 
hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
 ##
 @@ -0,0 +1,238 @@
+/*
+ * 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 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.Assert;
+import org.junit.Before;
+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 extends HiveMetastoreTest {
+
+  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();
+
+  @Rule
+  public ExpectedException exceptionRule = ExpectedException.none();
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation;
+
+  @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);
+exceptionRule.expectMessage("Metastore operation failed");
+
+Assert.assertFalse("Table should not exist", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+Transaction txn = catalog.newCreateTableTransaction(
+TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap());
+
+// create the table concurrently
+catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC);
+Assert.assertTrue("Table should be created", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+// expect the transaction to fail
+txn.commitTransaction();
 
 Review comment:
   Let me consume the test jar for iceberg-api in iceberg-hive then.


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 #357: Add in and not in predicates

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #357: Add in and not in predicates
URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r317373153
 
 

 ##
 File path: 
api/src/main/java/org/apache/iceberg/expressions/UnboundPredicate.java
 ##
 @@ -125,13 +149,40 @@ 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 = literals().stream().map(
+l -> {
+  Literal lit = l.to(field.type());
+  if (lit == null) {
+throw new ValidationException(String.format(
+"Invalid value for comparison inclusive type %s: %s (%s)",
+field.type(), l.value(), l.value().getClass().getName()));
+  }
+  return lit;
+})
+.filter(l -> l != Literals.aboveMax() && l != Literals.belowMin())
+.collect(Collectors.toSet());
+
+if (lits.isEmpty()) {
+  return Expressions.alwaysFalse();
+} else if (lits.size() == 1) {
+  return new BoundPredicate<>(Operation.EQ, new 
BoundReference<>(field.fieldId(),
+  schema.accessorForField(field.fieldId())), lits.iterator().next());
 
 Review comment:
   To get the literal, use `Iterables.getOnlyElement`


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 #357: Add in and not in predicates

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #357: Add in and not in predicates
URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r317373107
 
 

 ##
 File path: 
api/src/main/java/org/apache/iceberg/expressions/UnboundPredicate.java
 ##
 @@ -125,13 +149,40 @@ 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);
 
 Review comment:
   Please revert the indentation change on this line. It is causing this to get 
picked up in the changes, but I don't think it has actually changed.


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 #357: Add in and not in predicates

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #357: Add in and not in predicates
URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r317373006
 
 

 ##
 File path: api/src/main/java/org/apache/iceberg/expressions/Predicate.java
 ##
 @@ -39,37 +37,35 @@ public R ref() {
 return ref;
   }
 
-  public Literal literal() {
-return literal;
-  }
+  abstract String literalString();
 
   @Override
   public String toString() {
-switch (op) {
+switch (op()) {
 
 Review comment:
   This change is unnecessary, can you revert 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] aokolnychyi commented on a change in pull request #362: Support create and replace transactions in Catalog

2019-08-24 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_r317372959
 
 

 ##
 File path: core/src/main/java/org/apache/iceberg/BaseTransaction.java
 ##
 @@ -241,7 +234,14 @@ private void commitCreateTransaction() {
 }
   }
 
-  private void commitReplaceTransaction() {
+  private void commitReplaceTransaction(boolean orCreate) {
+if (base == null && !orCreate) {
+  throw new NoSuchTableException("Table doesn't exist");
+} else if (base == null) {
+  commitCreateTransaction();
 
 Review comment:
   Should we use `replaceMetadata` to get the number of retries if `base` is 
null?


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 #357: Add in and not in predicates

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #357: Add in and not in predicates
URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r317372831
 
 

 ##
 File path: 
api/src/main/java/org/apache/iceberg/expressions/BoundSetPredicate.java
 ##
 @@ -0,0 +1,176 @@
+/*
+ * 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.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.iceberg.util.CharSequenceWrapper;
+
+public class BoundSetPredicate extends Predicate> {
+  private final LiteralSet literalSet;
+
+  BoundSetPredicate(Operation op, BoundReference ref, Set> lits) 
{
+super(op, ref);
+Preconditions.checkArgument(op == Operation.IN || op == Operation.NOT_IN,
+"%s predicate does not support a set of literals", op);
+this.literalSet = new LiteralSet<>(lits);
+  }
+
+  BoundSetPredicate(Operation op, BoundReference ref, LiteralSet lits) {
+super(op, ref);
+Preconditions.checkArgument(op == Operation.IN || op == Operation.NOT_IN,
+"%s predicate does not support a literal set", op);
+this.literalSet = lits;
+  }
+
+  @Override
+  public Expression negate() {
+return new BoundSetPredicate<>(op().negate(), ref(), literalSet);
+  }
+
+  public Set literalSet() {
+return literalSet;
+  }
+
+  @Override
+  String literalString() {
+return literalSet.toString();
+  }
+
+  /**
+   * 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}
+   */
+  private static class LiteralSet implements Set, Serializable {
+private final Set values;
+
+@SuppressWarnings("unchecked")
+LiteralSet(Set> lits) {
+  Preconditions.checkArgument(lits == null || lits.size() > 1,
+  "The input literal set must include more than 1 element.");
+  values = ImmutableSet.builder().addAll(
+  lits.stream().map(
+  lit -> {
+if (lit instanceof Literals.StringLiteral) {
+  return (T) 
CharSequenceWrapper.wrap(((Literals.StringLiteral) lit).value());
+} else {
+  return lit.value();
+}
+  }
+  ).iterator()).build();
+}
+
+@Override
+public String toString() {
+  return Joiner.on(", ").join(values);
+}
+
+@Override
+public boolean contains(Object object) {
+  if (object instanceof CharSequence) {
+return values.contains(CharSequenceWrapper.wrap((CharSequence) 
object));
+  }
+  return values.contains(object);
+}
+
+@Override
+public int size() {
+  return values.size();
+}
+
+@Override
+public boolean isEmpty() {
+  return values.isEmpty();
+}
+
+@Override
+@SuppressWarnings("unchecked")
+public Iterator iterator() {
+  return values.stream().map(
+  val -> {
+if (val instanceof CharSequenceWrapper) {
+  return (T) ((CharSequenceWrapper) val).get();
+} else {
+  return val;
+}
+  }).iterator();
+}
+
+@Override
+public boolean containsAll(Collection c) {
+  throw new UnsupportedOperationException(
+  "LiteralSet currently only supports checking if a single item is 
contained in it.");
+}
+
+@Override
+public Object[] toArray() {
+  throw new UnsupportedOperationException(
+  "Please use iterator() to visit the elements in the set.");
+}
+
+@Override
+public  X[] toArray(X[] a) {
+  throw new UnsupportedOperationException(
+  "Please use iterator() to visit the elements in the set.");
+}
+
+@Override
+public boolean add(T t) {
+  throw new UnsupportedOperationException(
+  "The set is immutable and cannot add an element.");
+}
+
+@Override
+public boolean remove(Object o) {
+  throw new UnsupportedOperationException(
+  "The set is immutable an

[GitHub] [incubator-iceberg] rdblue merged pull request #387: Optimize overwrite and delete commits.

2019-08-24 Thread GitBox
rdblue merged pull request #387: Optimize overwrite and delete commits.
URL: https://github.com/apache/incubator-iceberg/pull/387
 
 
   


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 #357: Add in and not in predicates

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #357: Add in and not in predicates
URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r317372784
 
 

 ##
 File path: 
api/src/main/java/org/apache/iceberg/expressions/BoundSetPredicate.java
 ##
 @@ -0,0 +1,176 @@
+/*
+ * 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.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.iceberg.util.CharSequenceWrapper;
+
+public class BoundSetPredicate extends Predicate> {
+  private final LiteralSet literalSet;
+
+  BoundSetPredicate(Operation op, BoundReference ref, Set> lits) 
{
+super(op, ref);
+Preconditions.checkArgument(op == Operation.IN || op == Operation.NOT_IN,
+"%s predicate does not support a set of literals", op);
+this.literalSet = new LiteralSet<>(lits);
+  }
+
+  BoundSetPredicate(Operation op, BoundReference ref, LiteralSet lits) {
+super(op, ref);
+Preconditions.checkArgument(op == Operation.IN || op == Operation.NOT_IN,
+"%s predicate does not support a literal set", op);
+this.literalSet = lits;
+  }
+
+  @Override
+  public Expression negate() {
+return new BoundSetPredicate<>(op().negate(), ref(), literalSet);
+  }
+
+  public Set literalSet() {
+return literalSet;
+  }
+
+  @Override
+  String literalString() {
+return literalSet.toString();
+  }
+
+  /**
+   * 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}
+   */
+  private static class LiteralSet implements Set, Serializable {
+private final Set values;
+
+@SuppressWarnings("unchecked")
+LiteralSet(Set> lits) {
+  Preconditions.checkArgument(lits == null || lits.size() > 1,
+  "The input literal set must include more than 1 element.");
+  values = ImmutableSet.builder().addAll(
+  lits.stream().map(
+  lit -> {
+if (lit instanceof Literals.StringLiteral) {
+  return (T) 
CharSequenceWrapper.wrap(((Literals.StringLiteral) lit).value());
+} else {
+  return lit.value();
+}
+  }
+  ).iterator()).build();
+}
+
+@Override
+public String toString() {
+  return Joiner.on(", ").join(values);
+}
+
+@Override
+public boolean contains(Object object) {
 
 Review comment:
   `CharSeqLiteralSet` should override this. If the object isn't a 
`CharSequence` then it should return false. If it is, then it should use a 
thread-local wrapper:
   
   ```java
 private static final ThreadLocal wrapper =
 ThreadLocal.withInitial(() -> CharSequenceWrapper.wrap(null));
   
 @Override
 public boolean contains(Object object) {
   if (object instanceof CharSequence) {
 return values.contains(wrapper.get().set((CharSequence) object));
   }
   return false;
 }
   ```


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 opened a new issue #409: Optimize StreamingDelete

2019-08-24 Thread GitBox
aokolnychyi opened a new issue #409: Optimize StreamingDelete
URL: https://github.com/apache/incubator-iceberg/issues/409
 
 
   We should use optimized deletes in `StreamingDelete` from #387.


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 #357: Add in and not in predicates

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #357: Add in and not in predicates
URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r317372657
 
 

 ##
 File path: 
api/src/main/java/org/apache/iceberg/expressions/BoundSetPredicate.java
 ##
 @@ -0,0 +1,176 @@
+/*
+ * 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.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.iceberg.util.CharSequenceWrapper;
+
+public class BoundSetPredicate extends Predicate> {
+  private final LiteralSet literalSet;
+
+  BoundSetPredicate(Operation op, BoundReference ref, Set> lits) 
{
+super(op, ref);
+Preconditions.checkArgument(op == Operation.IN || op == Operation.NOT_IN,
+"%s predicate does not support a set of literals", op);
+this.literalSet = new LiteralSet<>(lits);
+  }
+
+  BoundSetPredicate(Operation op, BoundReference ref, LiteralSet lits) {
+super(op, ref);
+Preconditions.checkArgument(op == Operation.IN || op == Operation.NOT_IN,
+"%s predicate does not support a literal set", op);
+this.literalSet = lits;
+  }
+
+  @Override
+  public Expression negate() {
+return new BoundSetPredicate<>(op().negate(), ref(), literalSet);
+  }
+
+  public Set literalSet() {
+return literalSet;
+  }
+
+  @Override
+  String literalString() {
+return literalSet.toString();
+  }
+
+  /**
+   * 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}
+   */
+  private static class LiteralSet implements Set, Serializable {
 
 Review comment:
   I think it would be cleaner to have two implementations: `LiteralSet` and 
`CharSeqLiteralSet` that extends `LiteralSet`. There are too many methods here 
that check for `CharSeq` and change behavior.


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 #357: Add in and not in predicates

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #357: Add in and not in predicates
URL: https://github.com/apache/incubator-iceberg/pull/357#discussion_r317372625
 
 

 ##
 File path: 
api/src/main/java/org/apache/iceberg/expressions/ExpressionVisitors.java
 ##
 @@ -51,6 +53,10 @@ public R or(R leftResult, R rightResult) {
   return null;
 }
 
+public  R predicate(BoundSetPredicate pred) {
+  return null;
 
 Review comment:
   This should also throw `UnsupportedOperationException` to fail any 
implementation that is missing support.


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 #398: Push down StringStartsWith in Spark IcebergSource

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #398: Push down StringStartsWith 
in Spark IcebergSource
URL: https://github.com/apache/incubator-iceberg/pull/398#discussion_r317372354
 
 

 ##
 File path: 
api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java
 ##
 @@ -254,5 +258,43 @@ public Boolean or(Boolean leftResult, Boolean 
rightResult) {
 public  Boolean notIn(BoundReference ref, Literal lit) {
   return ROWS_MIGHT_MATCH;
 }
+
+@Override
+public  Boolean startsWith(BoundReference ref, Literal lit) {
+  int id = ref.fieldId();
+
+  String prefix = (String) lit.value();
+  ByteBuffer prefixAsBytes = 
ByteBuffer.wrap(prefix.getBytes(StandardCharsets.UTF_8));
 
 Review comment:
   Maybe we should add `toByteBuffer` and `fromByteBuffer` methods to 
`Literal`. That would make conversion to the single-value serialization easier 
(and public) and would solve this problem.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #362: Support create and replace transactions in Catalog

2019-08-24 Thread GitBox
rdblue 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_r317372326
 
 

 ##
 File path: core/src/main/java/org/apache/iceberg/BaseTransaction.java
 ##
 @@ -241,7 +234,14 @@ private void commitCreateTransaction() {
 }
   }
 
-  private void commitReplaceTransaction() {
+  private void commitReplaceTransaction(boolean orCreate) {
+if (base == null && !orCreate) {
+  throw new NoSuchTableException("Table doesn't exist");
 
 Review comment:
   Then I think we can remove this if/elseif/else block.


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-24 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_r317372229
 
 

 ##
 File path: core/src/main/java/org/apache/iceberg/BaseTransaction.java
 ##
 @@ -241,7 +234,14 @@ private void commitCreateTransaction() {
 }
   }
 
-  private void commitReplaceTransaction() {
+  private void commitReplaceTransaction(boolean orCreate) {
+if (base == null && !orCreate) {
+  throw new NoSuchTableException("Table doesn't exist");
 
 Review comment:
   Yeah, it is actually handled there as well. This should never happen. 


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 #362: Support create and replace transactions in Catalog

2019-08-24 Thread GitBox
rdblue 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_r317372191
 
 

 ##
 File path: 
hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
 ##
 @@ -0,0 +1,238 @@
+/*
+ * 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 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.Assert;
+import org.junit.Before;
+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 extends HiveMetastoreTest {
+
+  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();
+
+  @Rule
+  public ExpectedException exceptionRule = ExpectedException.none();
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation;
+
+  @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);
+exceptionRule.expectMessage("Metastore operation failed");
+
+Assert.assertFalse("Table should not exist", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+Transaction txn = catalog.newCreateTableTransaction(
+TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap());
+
+// create the table concurrently
+catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC);
+Assert.assertTrue("Table should be created", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+// expect the transaction to fail
+txn.commitTransaction();
+  }
+
+  @Test
+  public void testCreateTableTxnTableAlreadyExists() {
+exceptionRule.expect(AlreadyExistsException.class);
+exceptionRule.expectMessage("Table already exists: hivedb.tbl");
+
+Assert.assertFalse("Table should not exist", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+// create a table before starting a transaction
+catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC);
+Assert.assertTrue("Table should be created", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+// should not be possible to start a new create table transaction
+catalog.newCreateTableTransaction(
+TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap());
+  }
+
+  @Test
+  public void testReplaceTableTxn() {
+catalog.createTable(TABLE_IDENTIFIER, 

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #362: Support create and replace transactions in Catalog

2019-08-24 Thread GitBox
rdblue 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_r317372204
 
 

 ##
 File path: 
hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
 ##
 @@ -0,0 +1,238 @@
+/*
+ * 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 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.Assert;
+import org.junit.Before;
+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 extends HiveMetastoreTest {
+
+  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();
+
+  @Rule
+  public ExpectedException exceptionRule = ExpectedException.none();
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation;
+
+  @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);
+exceptionRule.expectMessage("Metastore operation failed");
+
+Assert.assertFalse("Table should not exist", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+Transaction txn = catalog.newCreateTableTransaction(
+TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap());
+
+// create the table concurrently
+catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC);
+Assert.assertTrue("Table should be created", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+// expect the transaction to fail
+txn.commitTransaction();
+  }
+
+  @Test
+  public void testCreateTableTxnTableAlreadyExists() {
+exceptionRule.expect(AlreadyExistsException.class);
+exceptionRule.expectMessage("Table already exists: hivedb.tbl");
+
+Assert.assertFalse("Table should not exist", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+// create a table before starting a transaction
+catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC);
+Assert.assertTrue("Table should be created", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+// should not be possible to start a new create table transaction
+catalog.newCreateTableTransaction(
+TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap());
+  }
+
+  @Test
+  public void testReplaceTableTxn() {
+catalog.createTable(TABLE_IDENTIFIER, 

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #362: Support create and replace transactions in Catalog

2019-08-24 Thread GitBox
rdblue 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_r317372149
 
 

 ##
 File path: 
hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
 ##
 @@ -0,0 +1,238 @@
+/*
+ * 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 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.Assert;
+import org.junit.Before;
+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 extends HiveMetastoreTest {
+
+  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();
+
+  @Rule
+  public ExpectedException exceptionRule = ExpectedException.none();
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation;
+
+  @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);
+exceptionRule.expectMessage("Metastore operation failed");
+
+Assert.assertFalse("Table should not exist", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+Transaction txn = catalog.newCreateTableTransaction(
+TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap());
+
+// create the table concurrently
+catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC);
+Assert.assertTrue("Table should be created", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+// expect the transaction to fail
+txn.commitTransaction();
+  }
+
+  @Test
+  public void testCreateTableTxnTableAlreadyExists() {
+exceptionRule.expect(AlreadyExistsException.class);
+exceptionRule.expectMessage("Table already exists: hivedb.tbl");
+
+Assert.assertFalse("Table should not exist", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+// create a table before starting a transaction
+catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC);
+Assert.assertTrue("Table should be created", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+// should not be possible to start a new create table transaction
+catalog.newCreateTableTransaction(
+TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap());
+  }
+
+  @Test
+  public void testReplaceTableTxn() {
+catalog.createTable(TABLE_IDENTIFIER, 

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #362: Support create and replace transactions in Catalog

2019-08-24 Thread GitBox
rdblue 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_r317372136
 
 

 ##
 File path: 
hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
 ##
 @@ -0,0 +1,238 @@
+/*
+ * 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 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.Assert;
+import org.junit.Before;
+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 extends HiveMetastoreTest {
+
+  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();
+
+  @Rule
+  public ExpectedException exceptionRule = ExpectedException.none();
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation;
+
+  @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);
+exceptionRule.expectMessage("Metastore operation failed");
+
+Assert.assertFalse("Table should not exist", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+Transaction txn = catalog.newCreateTableTransaction(
+TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap());
+
+// create the table concurrently
+catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC);
+Assert.assertTrue("Table should be created", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+// expect the transaction to fail
+txn.commitTransaction();
 
 Review comment:
   This is minor, so don't worry about it if it would be a lot of work to 
convert.


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 #362: Support create and replace transactions in Catalog

2019-08-24 Thread GitBox
rdblue 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_r317372108
 
 

 ##
 File path: 
hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
 ##
 @@ -0,0 +1,238 @@
+/*
+ * 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 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.Assert;
+import org.junit.Before;
+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 extends HiveMetastoreTest {
+
+  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();
+
+  @Rule
+  public ExpectedException exceptionRule = ExpectedException.none();
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String tableLocation;
+
+  @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);
+exceptionRule.expectMessage("Metastore operation failed");
+
+Assert.assertFalse("Table should not exist", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+Transaction txn = catalog.newCreateTableTransaction(
+TABLE_IDENTIFIER, SCHEMA, SPEC, tableLocation, Maps.newHashMap());
+
+// create the table concurrently
+catalog.createTable(TABLE_IDENTIFIER, SCHEMA, SPEC);
+Assert.assertTrue("Table should be created", 
catalog.tableExists(TABLE_IDENTIFIER));
+
+// expect the transaction to fail
+txn.commitTransaction();
 
 Review comment:
   I prefer to use `TestHelpers.assertThrows` for cases like this instead of 
the `ExpectedException` rule because I think what is happening is more clear in 
the test and so that you can make more assertions after the exception is thrown 
to validate that the conflicting table has not changed.


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...

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #362: Support create and replace transactions in Catalog

2019-08-24 Thread GitBox
rdblue 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_r317371864
 
 

 ##
 File path: core/src/main/java/org/apache/iceberg/BaseTransaction.java
 ##
 @@ -241,7 +234,14 @@ private void commitCreateTransaction() {
 }
   }
 
-  private void commitReplaceTransaction() {
+  private void commitReplaceTransaction(boolean orCreate) {
+if (base == null && !orCreate) {
+  throw new NoSuchTableException("Table doesn't exist");
+} else if (base == null) {
+  commitCreateTransaction();
 
 Review comment:
   I don't think this is correct because `commitCreateTransaction` won't 
succeed if the table is created concurrently, but create or replace should 
succeed.


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 #362: Support create and replace transactions in Catalog

2019-08-24 Thread GitBox
rdblue 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_r317371838
 
 

 ##
 File path: api/src/main/java/org/apache/iceberg/Transaction.java
 ##
 @@ -26,6 +26,14 @@
  * A transaction for performing multiple updates to a table.
  */
 public interface Transaction {
+
+  enum TransactionType {
 
 Review comment:
   I don't think it is necessary to move this to the public API since both 
Transactions and BaseTransaction are in core.


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 #362: Support create and replace transactions in Catalog

2019-08-24 Thread GitBox
rdblue 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_r317371809
 
 

 ##
 File path: core/src/main/java/org/apache/iceberg/BaseTransaction.java
 ##
 @@ -241,7 +234,14 @@ private void commitCreateTransaction() {
 }
   }
 
-  private void commitReplaceTransaction() {
+  private void commitReplaceTransaction(boolean orCreate) {
+if (base == null && !orCreate) {
+  throw new NoSuchTableException("Table doesn't exist");
 
 Review comment:
   Since this isn't called inside the commit retry loop, it will always depend 
on the initial state of the transaction, right? In that case, the problem 
should be caught when the transaction is created 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
For additional commands, e-mail: issues-h...@iceberg.apache.org



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #362: Support create and replace transactions in Catalog

2019-08-24 Thread GitBox
rdblue 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_r317371730
 
 

 ##
 File path: 
core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java
 ##
 @@ -66,6 +67,35 @@ public int currentVersion() {
 return version;
   }
 
+  @Override
+  public TableMetadata refresh() {
+try {
+  doRefresh();
+} catch (NoSuchTableException e) {
+  LOG.warn("Could not find the table during refresh, setting current 
metadata to null", e);
+  currentMetadata = null;
+  currentMetadataLocation = null;
+  version = -1;
+  shouldRefresh = false;
+  throw e;
+}
+return current();
+  }
+
+  protected void doRefresh() {
+throw new UnsupportedOperationException("Not implemented: doRefresh");
+  }
+
+  @Override
+  public void commit(TableMetadata base, TableMetadata metadata) {
+doCommit(base, metadata);
+requestRefresh();
 
 Review comment:
   Being able to ensure this is called is an unexpected benefit!


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 #394: Re-deploy web page to fix #333

2019-08-24 Thread GitBox
rdblue commented on issue #394: Re-deploy web page to fix #333
URL: 
https://github.com/apache/incubator-iceberg/issues/394#issuecomment-524573489
 
 
   I just redeployed. Looks like it works to me.


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 closed issue #394: Re-deploy web page to fix #333

2019-08-24 Thread GitBox
rdblue closed issue #394: Re-deploy web page to fix #333
URL: https://github.com/apache/incubator-iceberg/issues/394
 
 
   


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 #389: Add test cases

2019-08-24 Thread GitBox
rdblue commented on a change in pull request #389: Add test cases
URL: https://github.com/apache/incubator-iceberg/pull/389#discussion_r317371298
 
 

 ##
 File path: core/src/test/java/org/apache/iceberg/TestScanSummary.java
 ##
 @@ -32,7 +33,47 @@
 import static org.apache.iceberg.expressions.Expressions.lessThan;
 import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
 
-public class TestScanSummary {
+public class TestScanSummary extends TableTestBase {
+
+  @Test
+  public void testSnapshotTimeRangeValidation() {
+long t0 = System.currentTimeMillis();
+
+table.newAppend()
+.appendFile(FILE_A) // data_bucket=0
+.appendFile(FILE_B) // data_bucket=1
+.commit();
+
+long t1 = System.currentTimeMillis();
+
+table.newAppend()
+.appendFile(FILE_C) // data_bucket=2
+.commit();
+
+long secondSnapshotId = table.currentSnapshot().snapshotId();
+
+long t2 = System.currentTimeMillis();
+
+// expire the first snapshot
+table.expireSnapshots()
+.expireOlderThan(t1)
+.commit();
+
+Assert.assertEquals("Should have one snapshot",
+1, Lists.newArrayList(table.snapshots()).size());
 
 Review comment:
   Okay, I'll update it to make sure the timestamps aren't equal.


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 #408: Use null counts in metrics evaluators

2019-08-24 Thread GitBox
aokolnychyi commented on issue #408: Use null counts in metrics evaluators
URL: 
https://github.com/apache/incubator-iceberg/issues/408#issuecomment-524567397
 
 
   I'll 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 a change in pull request #398: Push down StringStartsWith in Spark IcebergSource

2019-08-24 Thread GitBox
aokolnychyi commented on a change in pull request #398: Push down 
StringStartsWith in Spark IcebergSource
URL: https://github.com/apache/incubator-iceberg/pull/398#discussion_r317369003
 
 

 ##
 File path: 
api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java
 ##
 @@ -254,5 +258,43 @@ public Boolean or(Boolean leftResult, Boolean 
rightResult) {
 public  Boolean notIn(BoundReference ref, Literal lit) {
   return ROWS_MIGHT_MATCH;
 }
+
+@Override
+public  Boolean startsWith(BoundReference ref, Literal lit) {
+  int id = ref.fieldId();
+
+  String prefix = (String) lit.value();
+  ByteBuffer prefixAsBytes = 
ByteBuffer.wrap(prefix.getBytes(StandardCharsets.UTF_8));
 
 Review comment:
   I would like to do a similar thing in `ParquetMetricsRowGroupFilter`. 
However, `StringLiteral` is not accessible there. Do we want to make `Literals` 
public?


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 opened a new pull request #401: Introduce a wait before each check lock state request to HMS

2019-08-24 Thread GitBox
aokolnychyi opened a new pull request #401: Introduce a wait before each check 
lock state request to HMS
URL: https://github.com/apache/incubator-iceberg/pull/401
 
 
   This PR introduces a wait between check lock state requests to HMS and 
partially resolves #366.
   
   The test was originally contributed by @rdsr.


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 closed pull request #401: Introduce a wait before each check lock state request to HMS

2019-08-24 Thread GitBox
aokolnychyi closed pull request #401: Introduce a wait before each check lock 
state request to HMS
URL: https://github.com/apache/incubator-iceberg/pull/401
 
 
   


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 #401: Introduce a wait before each check lock state request to HMS

2019-08-24 Thread GitBox
aokolnychyi commented on a change in pull request #401: Introduce a wait before 
each check lock state request to HMS
URL: https://github.com/apache/incubator-iceberg/pull/401#discussion_r317356685
 
 

 ##
 File path: core/src/main/java/org/apache/iceberg/TableProperties.java
 ##
 @@ -97,4 +97,7 @@ private TableProperties() {}
 
   public static final String WRITE_AUDIT_PUBLISH_ENABLED = "write.wap.enabled";
   public static final String WRITE_AUDIT_PUBLISH_ENABLED_DEFAULT = "false";
+
+  public static final String HIVE_CHECK_LOCK_STATE_WAIT_MS = 
"hive.lock.check-state-wait-ms";
 
 Review comment:
   Done


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
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 #362: Support create and replace transactions in Catalog

2019-08-24 Thread GitBox
aokolnychyi commented on issue #362: Support create and replace transactions in 
Catalog
URL: https://github.com/apache/incubator-iceberg/pull/362#issuecomment-524541950
 
 
   @rdblue ready for another review round


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 #389: Add test cases

2019-08-24 Thread GitBox
aokolnychyi commented on a change in pull request #389: Add test cases
URL: https://github.com/apache/incubator-iceberg/pull/389#discussion_r317353538
 
 

 ##
 File path: core/src/test/java/org/apache/iceberg/TestScanSummary.java
 ##
 @@ -32,7 +33,47 @@
 import static org.apache.iceberg.expressions.Expressions.lessThan;
 import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
 
-public class TestScanSummary {
+public class TestScanSummary extends TableTestBase {
+
+  @Test
+  public void testSnapshotTimeRangeValidation() {
+long t0 = System.currentTimeMillis();
+
+table.newAppend()
+.appendFile(FILE_A) // data_bucket=0
+.appendFile(FILE_B) // data_bucket=1
+.commit();
+
+long t1 = System.currentTimeMillis();
+
+table.newAppend()
+.appendFile(FILE_C) // data_bucket=2
+.commit();
+
+long secondSnapshotId = table.currentSnapshot().snapshotId();
+
+long t2 = System.currentTimeMillis();
+
+// expire the first snapshot
+table.expireSnapshots()
+.expireOlderThan(t1)
+.commit();
+
+Assert.assertEquals("Should have one snapshot",
+1, Lists.newArrayList(table.snapshots()).size());
 
 Review comment:
   ```
   java.lang.AssertionError: No exception was thrown (Should fail summary 
because range may include expired snapshots), expected: 
java.lang.IllegalArgumentException
   ```


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 #389: Add test cases

2019-08-24 Thread GitBox
aokolnychyi commented on a change in pull request #389: Add test cases
URL: https://github.com/apache/incubator-iceberg/pull/389#discussion_r317353512
 
 

 ##
 File path: core/src/test/java/org/apache/iceberg/TestScanSummary.java
 ##
 @@ -32,7 +33,47 @@
 import static org.apache.iceberg.expressions.Expressions.lessThan;
 import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
 
-public class TestScanSummary {
+public class TestScanSummary extends TableTestBase {
+
+  @Test
+  public void testSnapshotTimeRangeValidation() {
+long t0 = System.currentTimeMillis();
+
+table.newAppend()
+.appendFile(FILE_A) // data_bucket=0
+.appendFile(FILE_B) // data_bucket=1
+.commit();
+
+long t1 = System.currentTimeMillis();
+
+table.newAppend()
+.appendFile(FILE_C) // data_bucket=2
+.commit();
+
+long secondSnapshotId = table.currentSnapshot().snapshotId();
+
+long t2 = System.currentTimeMillis();
+
+// expire the first snapshot
+table.expireSnapshots()
+.expireOlderThan(t1)
+.commit();
+
+Assert.assertEquals("Should have one snapshot",
+1, Lists.newArrayList(table.snapshots()).size());
 
 Review comment:
   @xabriel @rdblue we also see failures from time to time in our internal 
builds


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