You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/07/01 18:04:45 UTC

[GitHub] [iceberg] rdblue opened a new pull request #1156: Add Spark 3 DDL tests

rdblue opened a new pull request #1156:
URL: https://github.com/apache/iceberg/pull/1156


   This adds tests for `CREATE TABLE` and `ALTER TABLE`.
   
   The tests primarily focus on the way that query info is passed to Iceberg. This is not intended to be exhaustive for either ways to express changes as SQL or for what Iceberg supports. For example, this only tests updating an int column to long to validate that `TableChange.UpdateColumnType` is handled.
   
   This PR includes #1149 because it adds the base test classes. Once that has been merged, I'll rebase this to remove that 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:
users@infra.apache.org



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


[GitHub] [iceberg] danielcweeks commented on pull request #1156: Add Spark 3 SQL tests

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on pull request #1156:
URL: https://github.com/apache/iceberg/pull/1156#issuecomment-655770188


   +1 Lots of great tests.  One minor comment.


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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1156: Add Spark 3 SQL tests

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1156:
URL: https://github.com/apache/iceberg/pull/1156#discussion_r451895218



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
##########
@@ -110,9 +110,17 @@ public Transaction newReplaceTableTransaction(
       throw new NoSuchTableException("No such table: " + identifier);
     }
 
-    String baseLocation = location != null ? location : defaultWarehouseLocation(identifier);
     Map<String, String> tableProperties = properties != null ? properties : Maps.newHashMap();
-    TableMetadata metadata = TableMetadata.newTableMetadata(schema, spec, baseLocation, tableProperties);
+
+    TableMetadata metadata;
+    if (ops.current() != null) {

Review comment:
       It would probably wipe out column comments. I'll have to fix that.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1156: Add Spark 3 SQL tests

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1156:
URL: https://github.com/apache/iceberg/pull/1156#discussion_r449194344



##########
File path: hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
##########
@@ -155,7 +155,7 @@ public void testReplaceTableTxn() {
     txn.commitTransaction();
 
     Table table = catalog.loadTable(TABLE_IDENTIFIER);
-    Assert.assertEquals("Partition spec should match", PartitionSpec.unpartitioned(), table.spec());
+    Assert.assertEquals("Partition spec should be unpartitioned", 0, table.spec().fields().size());

Review comment:
       I should note that this needed to be updated because the spec's ID didn't match, causing the test to fail.
   
   `PartitionSpec.unpartitioned()` doesn't necessarily have the right ID for a given table. We primarily use it when creating tables, where the spec gets rebuilt and assigned the right ID.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1156: Add Spark 3 SQL tests

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1156:
URL: https://github.com/apache/iceberg/pull/1156#discussion_r448703351



##########
File path: hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
##########
@@ -155,7 +155,7 @@ public void testReplaceTableTxn() {
     txn.commitTransaction();
 
     Table table = catalog.loadTable(TABLE_IDENTIFIER);
-    Assert.assertEquals("Partition spec should match", PartitionSpec.unpartitioned(), table.spec());
+    Assert.assertEquals("Partition spec should be unpartitioned", 0, table.spec().fields().size());

Review comment:
       Any partition spec with 0 fields is considered unpartitioned.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1156: Add Spark 3 SQL tests

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1156:
URL: https://github.com/apache/iceberg/pull/1156#discussion_r451866335



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
##########
@@ -110,9 +110,17 @@ public Transaction newReplaceTableTransaction(
       throw new NoSuchTableException("No such table: " + identifier);
     }
 
-    String baseLocation = location != null ? location : defaultWarehouseLocation(identifier);
     Map<String, String> tableProperties = properties != null ? properties : Maps.newHashMap();
-    TableMetadata metadata = TableMetadata.newTableMetadata(schema, spec, baseLocation, tableProperties);
+
+    TableMetadata metadata;
+    if (ops.current() != null) {

Review comment:
       I think this is correct because the `REPLACE TABLE` doesn't completely wipe out the old table. In most ways, it is the same table.
   
   Calling `buildReplacement` will replace the schema and partition spec so that the transaction can add a new snapshot. Table history, old snapshots, and existing table properties are kept so that you can inspect the table and don't need to add table configuration every time you run the SQL.
   
   We could add a flag to turn off this behavior and wipe out the old by default, but I don't think that's what users really want. It makes sense for things like default format and other settings to persist across replace operations, so that table configuration and table operations are orthogonal.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1156: Add Spark 3 SQL tests

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1156:
URL: https://github.com/apache/iceberg/pull/1156#discussion_r451729723



##########
File path: spark3/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java
##########
@@ -0,0 +1,322 @@
+/*
+ * 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.spark.sql;
+
+import java.util.Map;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.spark.SparkCatalogTestBase;
+import org.apache.iceberg.types.Types;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.spark.sql.functions.col;
+import static org.apache.spark.sql.functions.lit;
+import static org.apache.spark.sql.functions.when;
+
+public class TestCreateTableAsSelect extends SparkCatalogTestBase {
+
+  private final String sourceName;
+
+  public TestCreateTableAsSelect(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+    this.sourceName = tableName("source");
+
+    sql("CREATE TABLE IF NOT EXISTS %s (id bigint NOT NULL, data string) " +
+        "USING iceberg PARTITIONED BY (truncate(id, 3))", sourceName);
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", sourceName);
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void testUnpartitionedCTAS() {
+    sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", tableName, sourceName);
+
+    Schema expectedSchema = new Schema(
+        Types.NestedField.optional(1, "id", Types.LongType.get()),
+        Types.NestedField.optional(2, "data", Types.StringType.get())
+    );
+
+    Table ctasTable = validationCatalog.loadTable(tableIdent);
+
+    Assert.assertEquals("Should have expected nullable schema",
+        expectedSchema.asStruct(), ctasTable.schema().asStruct());
+    Assert.assertEquals("Should be an unpartitioned table",
+        0, ctasTable.spec().fields().size());
+    assertEquals("Should have rows matching the source table",
+        sql("SELECT * FROM %s ORDER BY id", sourceName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void testPartitionedCTAS() {
+    sql("CREATE TABLE %s USING iceberg PARTITIONED BY (id) AS SELECT * FROM %s ORDER BY id", tableName, sourceName);
+
+    Schema expectedSchema = new Schema(
+        Types.NestedField.optional(1, "id", Types.LongType.get()),
+        Types.NestedField.optional(2, "data", Types.StringType.get())
+    );
+
+    PartitionSpec expectedSpec = PartitionSpec.builderFor(expectedSchema)
+        .identity("id")
+        .build();
+
+    Table ctasTable = validationCatalog.loadTable(tableIdent);
+
+    Assert.assertEquals("Should have expected nullable schema",
+        expectedSchema.asStruct(), ctasTable.schema().asStruct());
+    Assert.assertEquals("Should be partitioned by id",
+        expectedSpec, ctasTable.spec());
+    assertEquals("Should have rows matching the source table",
+        sql("SELECT * FROM %s ORDER BY id", sourceName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+  }
+
+  @Test
+  public void testRTAS() {
+    sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", tableName, sourceName);
+
+    assertEquals("Should have rows matching the source table",
+        sql("SELECT * FROM %s ORDER BY id", sourceName),
+        sql("SELECT * FROM %s ORDER BY id", tableName));
+
+    sql("REPLACE TABLE %s USING iceberg PARTITIONED BY (part) AS " +
+        "SELECT id, data, CASE WHEN (id %% 2) = 0 THEN 'even' ELSE 'odd' END AS part " +
+        "FROM %s ORDER BY 3, 1", tableName, sourceName);
+
+    // spark_catalog does not use an atomic replace, so the table history and old spec is dropped
+    // the other catalogs do use atomic replace, so the spec id is incremented
+    boolean isAtomic = !"spark_catalog".equals(catalogName);

Review comment:
       This behavior is fixed in #1183. If this goes in first, I'll add the fix to that PR before merging.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] edgarRd commented on a change in pull request #1156: Add Spark 3 SQL tests

Posted by GitBox <gi...@apache.org>.
edgarRd commented on a change in pull request #1156:
URL: https://github.com/apache/iceberg/pull/1156#discussion_r448688284



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
##########
@@ -110,9 +110,17 @@ public Transaction newReplaceTableTransaction(
       throw new NoSuchTableException("No such table: " + identifier);
     }
 
-    String baseLocation = location != null ? location : defaultWarehouseLocation(identifier);
-    Map<String, String> tableProperties = properties != null ? properties : Maps.newHashMap();
-    TableMetadata metadata = TableMetadata.newTableMetadata(schema, spec, baseLocation, tableProperties);
+    TableMetadata metadata;
+    if (ops.current() != null) {
+      String baseLocation = location != null ? location : ops.current().location();
+      Map<String, String> tableProperties = properties != null ? properties : Maps.newHashMap();
+      metadata = ops.current().buildReplacement(schema, spec, baseLocation, tableProperties);
+    } else {
+      String baseLocation = location != null ? location : defaultWarehouseLocation(identifier);
+      Map<String, String> tableProperties = properties != null ? properties : Maps.newHashMap();

Review comment:
       You can probably take this out of the `if...else` since it's repeated and has to be done on either case.

##########
File path: hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
##########
@@ -155,7 +155,7 @@ public void testReplaceTableTxn() {
     txn.commitTransaction();
 
     Table table = catalog.loadTable(TABLE_IDENTIFIER);
-    Assert.assertEquals("Partition spec should match", PartitionSpec.unpartitioned(), table.spec());
+    Assert.assertEquals("Partition spec should be unpartitioned", 0, table.spec().fields().size());

Review comment:
       Is any `PartitionSpec`with `0` fields the `unpartitioned` spec or should it match a certain spec (like in the removed version)? Seems like the `equals` method takes `fields` and `specId` into the contract.

##########
File path: hive/src/main/java/org/apache/iceberg/hive/HiveCatalog.java
##########
@@ -347,6 +349,20 @@ protected boolean isValidIdentifier(TableIdentifier tableIdentifier) {
     return tableIdentifier.namespace().levels().length == 1;
   }
 
+  private TableIdentifier removeCatalogName(TableIdentifier to) {
+    if (isValidIdentifier(to)) {
+      return to;
+    }
+
+    // check if the identifier includes the catalog name and remove it
+    if (to.namespace().levels().length == 2 && name().equals(to.namespace().level(0))) {

Review comment:
       Is the catalog case sensitive? If not, `equalsIgnoreCase` may need to be used.

##########
File path: spark3/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.spark.sql;
+
+import java.io.File;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.spark.SparkCatalogTestBase;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestCreateTable extends SparkCatalogTestBase {
+  public TestCreateTable(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @After
+  public void dropTestTable() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void testCreateTable() {
+    Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent));
+
+    sql("CREATE TABLE %s (id BIGINT NOT NULL, data STRING) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    Assert.assertNotNull("Should load the new table", table);
+
+    StructType expectedSchema = StructType.of(
+        NestedField.required(1, "id", Types.LongType.get()),
+        NestedField.optional(2, "data", Types.StringType.get()));
+    Assert.assertEquals("Should have the expected schema", expectedSchema, table.schema().asStruct());
+    Assert.assertEquals("Should not be partitioned", 0, table.spec().fields().size());
+    Assert.assertNull("Should not have the default format set",
+        table.properties().get(TableProperties.DEFAULT_FILE_FORMAT));
+  }
+
+  @Test
+  public void testCreateTableUsingParquet() {

Review comment:
       Should we add tests for the other file formats?




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] danielcweeks commented on a change in pull request #1156: Add Spark 3 SQL tests

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on a change in pull request #1156:
URL: https://github.com/apache/iceberg/pull/1156#discussion_r451881098



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
##########
@@ -110,9 +110,17 @@ public Transaction newReplaceTableTransaction(
       throw new NoSuchTableException("No such table: " + identifier);
     }
 
-    String baseLocation = location != null ? location : defaultWarehouseLocation(identifier);
     Map<String, String> tableProperties = properties != null ? properties : Maps.newHashMap();
-    TableMetadata metadata = TableMetadata.newTableMetadata(schema, spec, baseLocation, tableProperties);
+
+    TableMetadata metadata;
+    if (ops.current() != null) {

Review comment:
       ok, I guess that makes sense.  I assumed the semantics to be the same a transactional `drop` and `create`, but based on a little searching, it's much less clear than that.  For example, with db2 create or replace will actually retain all data (assuming if aligns with the new table definition) by default.
   
   I agree that preserving table properties make sense, but wouldn't this wipe out comments (which we may also want to preserve) 




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1156: Add Spark 3 SQL tests

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1156:
URL: https://github.com/apache/iceberg/pull/1156#discussion_r448703745



##########
File path: spark3/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.spark.sql;
+
+import java.io.File;
+import java.util.Map;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.spark.SparkCatalogTestBase;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Test;
+
+public class TestCreateTable extends SparkCatalogTestBase {
+  public TestCreateTable(String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @After
+  public void dropTestTable() {
+    sql("DROP TABLE IF EXISTS %s", tableName);
+  }
+
+  @Test
+  public void testCreateTable() {
+    Assert.assertFalse("Table should not already exist", validationCatalog.tableExists(tableIdent));
+
+    sql("CREATE TABLE %s (id BIGINT NOT NULL, data STRING) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    Assert.assertNotNull("Should load the new table", table);
+
+    StructType expectedSchema = StructType.of(
+        NestedField.required(1, "id", Types.LongType.get()),
+        NestedField.optional(2, "data", Types.StringType.get()));
+    Assert.assertEquals("Should have the expected schema", expectedSchema, table.schema().asStruct());
+    Assert.assertEquals("Should not be partitioned", 0, table.spec().fields().size());
+    Assert.assertNull("Should not have the default format set",
+        table.properties().get(TableProperties.DEFAULT_FILE_FORMAT));
+  }
+
+  @Test
+  public void testCreateTableUsingParquet() {

Review comment:
       This is just a test that the provider is passed through correctly when using a specific catalog. I probably wouldn't add a test for other formats here in the SQL tests, but I would in unit tests for the SparkCatalog.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] rdblue merged pull request #1156: Add Spark 3 SQL tests

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1156:
URL: https://github.com/apache/iceberg/pull/1156


   


----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] danielcweeks commented on a change in pull request #1156: Add Spark 3 SQL tests

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on a change in pull request #1156:
URL: https://github.com/apache/iceberg/pull/1156#discussion_r451838028



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
##########
@@ -110,9 +110,17 @@ public Transaction newReplaceTableTransaction(
       throw new NoSuchTableException("No such table: " + identifier);
     }
 
-    String baseLocation = location != null ? location : defaultWarehouseLocation(identifier);
     Map<String, String> tableProperties = properties != null ? properties : Maps.newHashMap();
-    TableMetadata metadata = TableMetadata.newTableMetadata(schema, spec, baseLocation, tableProperties);
+
+    TableMetadata metadata;
+    if (ops.current() != null) {

Review comment:
       I'm not clear on whether this really should be the right behavior.  Basically we're saying that a replace table will keep the existing location (as opposed to using defaults).  I suspect we don't have create or replace with location semantics, but this is making some assumptions that a replacement is somehow the same as the old.  If we were to go with id based pathing convention, this wouldn't work.
   
   I don't think this is an issue at this point, but it might make sense to push this down to the location provider.




----------------------------------------------------------------
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:
users@infra.apache.org



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


[GitHub] [iceberg] edgarRd commented on a change in pull request #1156: Add Spark 3 SQL tests

Posted by GitBox <gi...@apache.org>.
edgarRd commented on a change in pull request #1156:
URL: https://github.com/apache/iceberg/pull/1156#discussion_r449263000



##########
File path: hive/src/test/java/org/apache/iceberg/hive/HiveCreateReplaceTableTest.java
##########
@@ -155,7 +155,7 @@ public void testReplaceTableTxn() {
     txn.commitTransaction();
 
     Table table = catalog.loadTable(TABLE_IDENTIFIER);
-    Assert.assertEquals("Partition spec should match", PartitionSpec.unpartitioned(), table.spec());
+    Assert.assertEquals("Partition spec should be unpartitioned", 0, table.spec().fields().size());

Review comment:
       Yeah, I supposed that was the reason for the change (spec's ID not matching). Thanks for the context!




----------------------------------------------------------------
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:
users@infra.apache.org



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