diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java index 3667c95394b7b..be86ac5476b75 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergAbstractMetadata.java @@ -68,7 +68,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; import io.airlift.slice.Slice; -import io.airlift.units.DataSize; import org.apache.hadoop.fs.Path; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.BaseTable; diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableProperties.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableProperties.java index 0f3d6fdb95344..b54497041a785 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableProperties.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergTableProperties.java @@ -16,23 +16,18 @@ import com.facebook.presto.common.type.ArrayType; import com.facebook.presto.spi.session.PropertyMetadata; import com.google.common.collect.ImmutableList; -import io.airlift.units.DataSize; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.TableProperties; import javax.inject.Inject; -import java.lang.reflect.AccessibleObject; -import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Set; import static com.facebook.presto.common.type.VarcharType.VARCHAR; import static com.facebook.presto.common.type.VarcharType.createUnboundedVarcharType; import static com.facebook.presto.spi.session.PropertyMetadata.booleanProperty; -import static com.facebook.presto.spi.session.PropertyMetadata.dataSizeProperty; import static com.facebook.presto.spi.session.PropertyMetadata.integerProperty; import static com.facebook.presto.spi.session.PropertyMetadata.longProperty; import static com.facebook.presto.spi.session.PropertyMetadata.stringProperty; diff --git a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java index 7547d2d449946..1cc7278e84040 100644 --- a/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java +++ b/presto-iceberg/src/main/java/com/facebook/presto/iceberg/IcebergUtil.java @@ -132,7 +132,6 @@ import static com.facebook.presto.iceberg.ExpressionConverter.toIcebergExpression; import static com.facebook.presto.iceberg.FileContent.POSITION_DELETES; import static com.facebook.presto.iceberg.FileContent.fromIcebergFileContent; -import static com.facebook.presto.iceberg.FileFormat.PARQUET; import static com.facebook.presto.iceberg.IcebergColumnHandle.DATA_SEQUENCE_NUMBER_COLUMN_HANDLE; import static com.facebook.presto.iceberg.IcebergColumnHandle.PATH_COLUMN_HANDLE; import static com.facebook.presto.iceberg.IcebergErrorCode.ICEBERG_INVALID_FORMAT_VERSION; @@ -1269,7 +1268,8 @@ public static String dataLocation(Table icebergTable) return dataLocation; } - public static Long getSplitSize(Table table) { + public static Long getSplitSize(Table table) + { return Long.parseLong(table.properties() .getOrDefault(SPLIT_SIZE, String.valueOf(SPLIT_SIZE_DEFAULT))); diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedSmokeTestBase.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedSmokeTestBase.java index 074a36e43ebb8..33b8103ddd0fe 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedSmokeTestBase.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/IcebergDistributedSmokeTestBase.java @@ -146,7 +146,8 @@ public void testShowCreateTable() " location = '%s',\n" + " metadata_delete_after_commit = false,\n" + " metadata_previous_versions_max = 100,\n" + - " metrics_max_inferred_column = 100\n" + + " metrics_max_inferred_column = 100,\n" + + " \"read.split.target-size\" = 134217728\n" + ")", schemaName, getLocation(schemaName, "orders"))); } @@ -427,7 +428,8 @@ protected void testCreatePartitionedTableAs(Session session, FileFormat fileForm " metadata_delete_after_commit = false,\n" + " metadata_previous_versions_max = 100,\n" + " metrics_max_inferred_column = 100,\n" + - " partitioning = ARRAY['order_status','ship_priority','bucket(order_key, 9)']\n" + + " partitioning = ARRAY['order_status','ship_priority','bucket(order_key, 9)'],\n" + + " \"read.split.target-size\" = 134217728\n" + ")", getSession().getCatalog().get(), getSession().getSchema().get(), @@ -630,7 +632,8 @@ public void testTableComments() " location = '%s',\n" + " metadata_delete_after_commit = false,\n" + " metadata_previous_versions_max = 100,\n" + - " metrics_max_inferred_column = 100\n" + + " metrics_max_inferred_column = 100,\n" + + " \"read.split.target-size\" = 134217728\n" + ")"; String createTableSql = format(createTableTemplate, schemaName, "test table comment", getLocation(schemaName, "test_table_comments")); @@ -722,7 +725,8 @@ private void testCreateTableLike() " metadata_delete_after_commit = false,\n" + " metadata_previous_versions_max = 100,\n" + " metrics_max_inferred_column = 100,\n" + - " partitioning = ARRAY['adate']\n" + + " partitioning = ARRAY['adate'],\n" + + " \"read.split.target-size\" = 134217728\n" + ")", getLocation(schemaName, "test_create_table_like_original"))); assertUpdate(session, "CREATE TABLE test_create_table_like_copy0 (LIKE test_create_table_like_original, col2 INTEGER)"); @@ -738,7 +742,8 @@ private void testCreateTableLike() " location = '%s',\n" + " metadata_delete_after_commit = false,\n" + " metadata_previous_versions_max = 100,\n" + - " metrics_max_inferred_column = 100\n" + + " metrics_max_inferred_column = 100,\n" + + " \"read.split.target-size\" = 134217728\n" + ")", getLocation(schemaName, "test_create_table_like_copy1"))); dropTable(session, "test_create_table_like_copy1"); @@ -750,7 +755,8 @@ private void testCreateTableLike() " location = '%s',\n" + " metadata_delete_after_commit = false,\n" + " metadata_previous_versions_max = 100,\n" + - " metrics_max_inferred_column = 100\n" + + " metrics_max_inferred_column = 100,\n" + + " \"read.split.target-size\" = 134217728\n" + ")", getLocation(schemaName, "test_create_table_like_copy2"))); dropTable(session, "test_create_table_like_copy2"); @@ -764,7 +770,8 @@ private void testCreateTableLike() " metadata_delete_after_commit = false,\n" + " metadata_previous_versions_max = 100,\n" + " metrics_max_inferred_column = 100,\n" + - " partitioning = ARRAY['adate']\n" + + " partitioning = ARRAY['adate'],\n" + + " \"read.split.target-size\" = 134217728\n" + ")", getLocation(schemaName, "test_create_table_like_original"))); dropTable(session, "test_create_table_like_copy3"); @@ -778,7 +785,8 @@ private void testCreateTableLike() " metadata_delete_after_commit = false,\n" + " metadata_previous_versions_max = 100,\n" + " metrics_max_inferred_column = 100,\n" + - " partitioning = ARRAY['adate']\n" + + " partitioning = ARRAY['adate'],\n" + + " \"read.split.target-size\" = 134217728\n" + ")", getLocation(schemaName, "test_create_table_like_original"))); dropTable(session, "test_create_table_like_copy4"); @@ -794,7 +802,8 @@ private void testCreateTableLike() " metadata_delete_after_commit = false,\n" + " metadata_previous_versions_max = 100,\n" + " metrics_max_inferred_column = 100,\n" + - " partitioning = ARRAY['adate']\n" + + " partitioning = ARRAY['adate'],\n" + + " \"read.split.target-size\" = 134217728\n" + ")", getLocation(schemaName, "test_create_table_like_copy5"))); dropTable(session, "test_create_table_like_copy5"); @@ -841,7 +850,8 @@ protected void testCreateTableWithFormatVersion(String formatVersion, String def " location = '%s',\n" + " metadata_delete_after_commit = false,\n" + " metadata_previous_versions_max = 100,\n" + - " metrics_max_inferred_column = 100\n" + + " metrics_max_inferred_column = 100,\n" + + " \"read.split.target-size\" = 134217728\n" + ")", getSession().getCatalog().get(), getSession().getSchema().get(), diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergSplitManager.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergSplitManager.java index a0b4294e2b735..73047fb4c9f00 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergSplitManager.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergSplitManager.java @@ -16,8 +16,6 @@ import com.facebook.presto.Session; import com.facebook.presto.common.transaction.TransactionId; import com.facebook.presto.execution.Lifespan; -import com.facebook.presto.execution.ScheduledSplit; -import com.facebook.presto.metadata.Split; import com.facebook.presto.spi.TableHandle; import com.facebook.presto.spi.WarningCollector; import com.facebook.presto.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy; @@ -31,7 +29,6 @@ import com.facebook.presto.testing.QueryRunner; import com.facebook.presto.tests.AbstractTestQueryFramework; import com.facebook.presto.transaction.TransactionManager; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.iceberg.TableProperties; import org.testng.annotations.Test; @@ -48,7 +45,6 @@ import static com.facebook.presto.iceberg.IcebergSessionProperties.TARGET_SPLIT_SIZE; import static com.facebook.presto.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergSystemTables.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergSystemTables.java index 15d09ddfd9bc0..8639aec75759a 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergSystemTables.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/TestIcebergSystemTables.java @@ -23,6 +23,7 @@ import com.facebook.presto.tests.DistributedQueryRunner; import com.facebook.presto.transaction.TransactionManager; import com.google.common.collect.ImmutableMap; +import io.airlift.units.DataSize; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; @@ -258,11 +259,11 @@ protected void checkTableProperties(String tableName, String deleteMode) { assertQuery(String.format("SHOW COLUMNS FROM test_schema.\"%s$properties\"", tableName), "VALUES ('key', 'varchar', '', '')," + "('value', 'varchar', '', '')"); - assertQuery(String.format("SELECT COUNT(*) FROM test_schema.\"%s$properties\"", tableName), "VALUES 7"); + assertQuery(String.format("SELECT COUNT(*) FROM test_schema.\"%s$properties\"", tableName), "VALUES 8"); List materializedRows = computeActual(getSession(), String.format("SELECT * FROM test_schema.\"%s$properties\"", tableName)).getMaterializedRows(); - assertThat(materializedRows).hasSize(7); + assertThat(materializedRows).hasSize(8); assertThat(materializedRows) .anySatisfy(row -> assertThat(row) .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, "write.delete.mode", deleteMode))) @@ -277,18 +278,20 @@ protected void checkTableProperties(String tableName, String deleteMode) .anySatisfy(row -> assertThat(row) .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, "write.metadata.delete-after-commit.enabled", "false"))) .anySatisfy(row -> assertThat(row) - .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, "write.metadata.metrics.max-inferred-column-defaults", "100"))); + .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, "write.metadata.metrics.max-inferred-column-defaults", "100"))) + .anySatisfy(row -> assertThat(row) + .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, IcebergTableProperties.TARGET_SPLIT_SIZE, Long.toString(DataSize.valueOf("128MB").toBytes())))); } protected void checkORCFormatTableProperties(String tableName, String deleteMode) { assertQuery(String.format("SHOW COLUMNS FROM test_schema.\"%s$properties\"", tableName), "VALUES ('key', 'varchar', '', '')," + "('value', 'varchar', '', '')"); - assertQuery(String.format("SELECT COUNT(*) FROM test_schema.\"%s$properties\"", tableName), "VALUES 8"); + assertQuery(String.format("SELECT COUNT(*) FROM test_schema.\"%s$properties\"", tableName), "VALUES 9"); List materializedRows = computeActual(getSession(), String.format("SELECT * FROM test_schema.\"%s$properties\"", tableName)).getMaterializedRows(); - assertThat(materializedRows).hasSize(8); + assertThat(materializedRows).hasSize(9); assertThat(materializedRows) .anySatisfy(row -> assertThat(row) .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, "write.delete.mode", deleteMode))) @@ -305,7 +308,9 @@ protected void checkORCFormatTableProperties(String tableName, String deleteMode .anySatisfy(row -> assertThat(row) .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, "write.metadata.delete-after-commit.enabled", "false"))) .anySatisfy(row -> assertThat(row) - .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, "write.metadata.metrics.max-inferred-column-defaults", "100"))); + .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, "write.metadata.metrics.max-inferred-column-defaults", "100"))) + .anySatisfy(row -> assertThat(row) + .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, IcebergTableProperties.TARGET_SPLIT_SIZE, Long.toString(DataSize.valueOf("128MB").toBytes())))); } @Test diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/nessie/TestIcebergSystemTablesNessie.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/nessie/TestIcebergSystemTablesNessie.java index 1ddc4c3235e48..d4069c4b78de1 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/nessie/TestIcebergSystemTablesNessie.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/nessie/TestIcebergSystemTablesNessie.java @@ -16,6 +16,7 @@ import com.facebook.presto.Session; import com.facebook.presto.iceberg.IcebergConfig; import com.facebook.presto.iceberg.IcebergPlugin; +import com.facebook.presto.iceberg.IcebergTableProperties; import com.facebook.presto.iceberg.TestIcebergSystemTables; import com.facebook.presto.testing.MaterializedResult; import com.facebook.presto.testing.MaterializedRow; @@ -23,6 +24,7 @@ import com.facebook.presto.testing.containers.NessieContainer; import com.facebook.presto.tests.DistributedQueryRunner; import com.google.common.collect.ImmutableMap; +import io.airlift.units.DataSize; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; @@ -89,11 +91,11 @@ protected void checkTableProperties(String tableName, String deleteMode) { assertQuery(String.format("SHOW COLUMNS FROM test_schema.\"%s$properties\"", tableName), "VALUES ('key', 'varchar', '', '')," + "('value', 'varchar', '', '')"); - assertQuery(String.format("SELECT COUNT(*) FROM test_schema.\"%s$properties\"", tableName), "VALUES 9"); + assertQuery(String.format("SELECT COUNT(*) FROM test_schema.\"%s$properties\"", tableName), "VALUES 10"); List materializedRows = computeActual(getSession(), String.format("SELECT * FROM test_schema.\"%s$properties\"", tableName)).getMaterializedRows(); - assertThat(materializedRows).hasSize(9); + assertThat(materializedRows).hasSize(10); assertThat(materializedRows) .anySatisfy(row -> assertThat(row) .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, "write.delete.mode", deleteMode))) @@ -110,7 +112,9 @@ protected void checkTableProperties(String tableName, String deleteMode) .anySatisfy(row -> assertThat(row) .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, "write.metadata.previous-versions-max", "100"))) .anySatisfy(row -> assertThat(row) - .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, "write.metadata.metrics.max-inferred-column-defaults", "100"))); + .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, "write.metadata.metrics.max-inferred-column-defaults", "100"))) + .anySatisfy(row -> assertThat(row) + .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, IcebergTableProperties.TARGET_SPLIT_SIZE, Long.toString(DataSize.valueOf("128MB").toBytes())))); } @Override @@ -118,11 +122,11 @@ protected void checkORCFormatTableProperties(String tableName, String deleteMode { assertQuery(String.format("SHOW COLUMNS FROM test_schema.\"%s$properties\"", tableName), "VALUES ('key', 'varchar', '', '')," + "('value', 'varchar', '', '')"); - assertQuery(String.format("SELECT COUNT(*) FROM test_schema.\"%s$properties\"", tableName), "VALUES 10"); + assertQuery(String.format("SELECT COUNT(*) FROM test_schema.\"%s$properties\"", tableName), "VALUES 11"); List materializedRows = computeActual(getSession(), String.format("SELECT * FROM test_schema.\"%s$properties\"", tableName)).getMaterializedRows(); - assertThat(materializedRows).hasSize(10); + assertThat(materializedRows).hasSize(11); assertThat(materializedRows) .anySatisfy(row -> assertThat(row) .isEqualTo(new MaterializedRow(MaterializedResult.DEFAULT_PRECISION, "write.delete.mode", deleteMode))) diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/procedure/TestSetTablePropertyProcedure.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/procedure/TestSetTablePropertyProcedure.java index a6ff137f8bc32..45e9a56ab3e28 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/procedure/TestSetTablePropertyProcedure.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/procedure/TestSetTablePropertyProcedure.java @@ -64,21 +64,21 @@ public void testSetTablePropertyProcedurePositionalArgs() String tableName = "table_property_table_test"; createTable(tableName); try { - String propertyKey = "read.split.target-size"; + String propertyKey = "read.split.planning-lookback"; String propertyValue = "268435456"; assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'a')", 1); Table table = loadTable(tableName); table.refresh(); - assertEquals(table.properties().size(), 7); + assertEquals(table.properties().size(), 8); assertEquals(table.properties().get(propertyKey), null); assertUpdate(format("CALL system.set_table_property('%s', '%s', '%s', '%s')", TEST_SCHEMA, tableName, propertyKey, propertyValue)); table.refresh(); // now the table property read.split.target-size should have new value - assertEquals(table.properties().size(), 8); + assertEquals(table.properties().size(), 9); assertEquals(table.properties().get(propertyKey), propertyValue); } finally { @@ -92,14 +92,14 @@ public void testSetTablePropertyProcedureNamedArgs() String tableName = "table_property_table_arg_test"; createTable(tableName); try { - String propertyKey = "read.split.target-size"; + String propertyKey = "read.split.planning-lookback"; String propertyValue = "268435456"; assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'a')", 1); Table table = loadTable(tableName); table.refresh(); - assertEquals(table.properties().size(), 7); + assertEquals(table.properties().size(), 8); assertEquals(table.properties().get(propertyKey), null); assertUpdate(format("CALL system.set_table_property(schema => '%s', key => '%s', value => '%s', table_name => '%s')", @@ -107,7 +107,7 @@ public void testSetTablePropertyProcedureNamedArgs() table.refresh(); // now the table property read.split.target-size should have new value - assertEquals(table.properties().size(), 8); + assertEquals(table.properties().size(), 9); assertEquals(table.properties().get(propertyKey), propertyValue); } finally { @@ -129,14 +129,14 @@ public void testSetTablePropertyProcedureUpdateExisting() Table table = loadTable(tableName); table.refresh(); - assertEquals(table.properties().size(), 7); + assertEquals(table.properties().size(), 8); assertEquals(table.properties().get(propertyKey), "4"); assertUpdate(format("CALL system.set_table_property('%s', '%s', '%s', '%s')", TEST_SCHEMA, tableName, propertyKey, propertyValue)); table.refresh(); // now the table property commit.retry.num-retries should have new value - assertEquals(table.properties().size(), 7); + assertEquals(table.properties().size(), 8); assertEquals(table.properties().get(propertyKey), propertyValue); } finally { diff --git a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/rest/TestIcebergSmokeRestNestedNamespace.java b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/rest/TestIcebergSmokeRestNestedNamespace.java index 1613bcfd9ba10..969b5b5687c47 100644 --- a/presto-iceberg/src/test/java/com/facebook/presto/iceberg/rest/TestIcebergSmokeRestNestedNamespace.java +++ b/presto-iceberg/src/test/java/com/facebook/presto/iceberg/rest/TestIcebergSmokeRestNestedNamespace.java @@ -180,7 +180,8 @@ public void testShowCreateTable() " location = '%s',\n" + " metadata_delete_after_commit = false,\n" + " metadata_previous_versions_max = 100,\n" + - " metrics_max_inferred_column = 100\n" + + " metrics_max_inferred_column = 100,\n" + + " \"read.split.target-size\" = 134217728\n" + ")", schemaName, getLocation(schemaName, "orders"))); } @@ -215,7 +216,8 @@ public void testTableComments() " location = '%s',\n" + " metadata_delete_after_commit = false,\n" + " metadata_previous_versions_max = 100,\n" + - " metrics_max_inferred_column = 100\n" + + " metrics_max_inferred_column = 100,\n" + + " \"read.split.target-size\" = 134217728\n" + ")"; String createTableSql = format(createTableTemplate, schemaName, "test table comment", getLocation(schemaName, "test_table_comments")); @@ -255,7 +257,8 @@ protected void testCreatePartitionedTableAs(Session session, FileFormat fileForm " metadata_delete_after_commit = false,\n" + " metadata_previous_versions_max = 100,\n" + " metrics_max_inferred_column = 100,\n" + - " partitioning = ARRAY['order_status','ship_priority','bucket(order_key, 9)']\n" + + " partitioning = ARRAY['order_status','ship_priority','bucket(order_key, 9)'],\n" + + " \"read.split.target-size\" = 134217728\n" + ")", getSession().getCatalog().get(), getSession().getSchema().get(), @@ -315,7 +318,8 @@ protected void testCreateTableWithFormatVersion(String formatVersion, String def " location = '%s',\n" + " metadata_delete_after_commit = false,\n" + " metadata_previous_versions_max = 100,\n" + - " metrics_max_inferred_column = 100\n" + + " metrics_max_inferred_column = 100,\n" + + " \"read.split.target-size\" = 134217728\n" + ")", getSession().getCatalog().get(), getSession().getSchema().get(),