Skip to content

Commit

Permalink
Fix style and tests
Browse files Browse the repository at this point in the history
  • Loading branch information
ZacBlanco committed Jan 27, 2025
1 parent 2826a04 commit 4a07851
Show file tree
Hide file tree
Showing 9 changed files with 58 additions and 45 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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)));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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")));
}

Expand Down Expand Up @@ -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(),
Expand Down Expand Up @@ -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"));

Expand Down Expand Up @@ -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)");
Expand All @@ -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");

Expand All @@ -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");

Expand All @@ -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");
Expand All @@ -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");
Expand All @@ -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");
Expand Down Expand Up @@ -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(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand All @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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<MaterializedRow> 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)))
Expand All @@ -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<MaterializedRow> 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)))
Expand All @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,15 @@
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;
import com.facebook.presto.testing.QueryRunner;
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;

Expand Down Expand Up @@ -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<MaterializedRow> 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)))
Expand All @@ -110,19 +112,21 @@ 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
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<MaterializedRow> 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)))
Expand Down
Loading

0 comments on commit 4a07851

Please sign in to comment.