Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[WIP] Round trip tests #252

Draft
wants to merge 1 commit into
base: main
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
33 changes: 33 additions & 0 deletions core/src/test/java/io/onetable/GenericTable.java
Original file line number Diff line number Diff line change
Expand Up @@ -125,6 +125,39 @@ static GenericTable getInstanceWithCustomPartitionConfig(
}
}

static GenericTable getInstanceWithCommonSchema(
String tableName,
Path tempDir,
SparkSession sparkSession,
JavaSparkContext jsc,
TableFormat sourceFormat) {
switch (sourceFormat) {
case HUDI:
return TestSparkHudiTable.forGivenSchemaAndPartitioning(
tableName,
tempDir,
jsc,
TestCommonTableHelper.getCommonSchemaInAvroFormat(),
TestCommonTableHelper.getHudiPartitionConfig());
case DELTA:
return TestSparkDeltaTable.forGivenSchemaAndPartitioning(
tableName,
tempDir,
sparkSession,
TestCommonTableHelper.getCommonSchemaInStructFormat(),
TestCommonTableHelper.getDeltaPartitionField());
case ICEBERG:
return TestIcebergTable.forGivenSchemaAndPartitioning(
tableName,
tempDir,
jsc.hadoopConfiguration(),
TestCommonTableHelper.getCommonSchemaInIcebergFormat(),
TestCommonTableHelper.getIcebergPartitionField());
default:
throw new IllegalArgumentException("Unsupported source format: " + sourceFormat);
}
}

static String getTableName() {
return "test_table_" + UUID.randomUUID().toString().replaceAll("-", "_");
}
Expand Down
142 changes: 142 additions & 0 deletions core/src/test/java/io/onetable/ITOneTableClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -670,6 +670,148 @@ public void testMetadataRetention() throws Exception {
}
}

@Test
public void roundTripTesting() {
SourceClientProvider<?> hudiSourceClientProvider = getSourceClientProvider(TableFormat.HUDI);
SourceClientProvider<?> icebergSourceClientProvider =
getSourceClientProvider(TableFormat.ICEBERG);
SourceClientProvider<?> deltaSourceClientProvider = getSourceClientProvider(TableFormat.DELTA);
String onetablePartitionConfig = "level:VALUE";
SyncMode syncMode = SyncMode.INCREMENTAL;

// Create table in hudi and sync to iceberg and delta.
String tableName = getTableName();
OneTableClient oneTableClient = new OneTableClient(jsc.hadoopConfiguration());
List<?> insertRecords;
try (GenericTable table =
GenericTable.getInstanceWithCommonSchema(
tableName, tempDir, sparkSession, jsc, TableFormat.HUDI)) {
insertRecords = table.insertRows(100);

PerTableConfig perTableConfig =
PerTableConfig.builder()
.tableName(tableName)
.targetTableFormats(Arrays.asList(TableFormat.ICEBERG, TableFormat.DELTA))
.tableBasePath(table.getBasePath())
.tableDataPath(table.getDataPath())
.hudiSourceConfig(
HudiSourceConfig.builder()
.partitionFieldSpecConfig(onetablePartitionConfig)
.build())
.syncMode(syncMode)
.build();
oneTableClient.sync(perTableConfig, hudiSourceClientProvider);
checkDatasetEquivalence(
TableFormat.HUDI, table, Arrays.asList(TableFormat.ICEBERG, TableFormat.DELTA), 100);

// make multiple commits and then sync
table.insertRows(100);
table.upsertRows(insertRecords.subList(0, 20));
oneTableClient.sync(perTableConfig, hudiSourceClientProvider);
checkDatasetEquivalence(
TableFormat.HUDI, table, Arrays.asList(TableFormat.DELTA, TableFormat.ICEBERG), 200);
}

// Now Get the table as Delta and do Sync to Hudi and Iceberg.
try (GenericTable table =
GenericTable.getInstanceWithCommonSchema(
tableName, tempDir, sparkSession, jsc, TableFormat.DELTA)) {
PerTableConfig perTableConfig =
PerTableConfig.builder()
.tableName(tableName)
.targetTableFormats(Arrays.asList(TableFormat.HUDI, TableFormat.ICEBERG))
.tableBasePath(table.getBasePath())
.tableDataPath(table.getDataPath())
.hudiSourceConfig(
HudiSourceConfig.builder()
.partitionFieldSpecConfig(onetablePartitionConfig)
.build())
.syncMode(syncMode)
.build();

table.insertRows(100);
oneTableClient.sync(perTableConfig, deltaSourceClientProvider);
checkDatasetEquivalence(
TableFormat.DELTA, table, Arrays.asList(TableFormat.HUDI, TableFormat.ICEBERG), 300);

table.insertRows(100);
oneTableClient.sync(perTableConfig, deltaSourceClientProvider);
checkDatasetEquivalence(
TableFormat.DELTA, table, Arrays.asList(TableFormat.HUDI, TableFormat.ICEBERG), 400);

table.upsertRows(insertRecords.subList(0, 20));
oneTableClient.sync(perTableConfig, deltaSourceClientProvider);
checkDatasetEquivalence(
TableFormat.DELTA, table, Arrays.asList(TableFormat.HUDI, TableFormat.ICEBERG), 400);
}

// Get table as Iceberg and do Sync to Hudi and Delta.
try (GenericTable table =
GenericTable.getInstanceWithCommonSchema(
tableName, tempDir, sparkSession, jsc, TableFormat.ICEBERG)) {
PerTableConfig perTableConfig =
PerTableConfig.builder()
.tableName(tableName)
.targetTableFormats(Arrays.asList(TableFormat.HUDI, TableFormat.DELTA))
.tableBasePath(table.getBasePath())
.tableDataPath(table.getDataPath())
.hudiSourceConfig(
HudiSourceConfig.builder()
.partitionFieldSpecConfig(onetablePartitionConfig)
.build())
.syncMode(syncMode)
.build();

table.insertRows(100);
oneTableClient.sync(perTableConfig, icebergSourceClientProvider);
checkDatasetEquivalence(
TableFormat.ICEBERG, table, Arrays.asList(TableFormat.HUDI, TableFormat.DELTA), 500);

table.insertRows(100);
oneTableClient.sync(perTableConfig, icebergSourceClientProvider);
checkDatasetEquivalence(
TableFormat.ICEBERG, table, Arrays.asList(TableFormat.HUDI, TableFormat.DELTA), 600);

table.upsertRows(insertRecords.subList(0, 20));
oneTableClient.sync(perTableConfig, icebergSourceClientProvider);
checkDatasetEquivalence(
TableFormat.ICEBERG, table, Arrays.asList(TableFormat.HUDI, TableFormat.DELTA), 600);
}

// Get one last time, table as Hudi and do Sync to Iceberg and Delta.
try (GenericTable table =
GenericTable.getInstanceWithCommonSchema(
tableName, tempDir, sparkSession, jsc, TableFormat.HUDI)) {
PerTableConfig perTableConfig =
PerTableConfig.builder()
.tableName(tableName)
.targetTableFormats(Arrays.asList(TableFormat.ICEBERG, TableFormat.DELTA))
.tableBasePath(table.getBasePath())
.tableDataPath(table.getDataPath())
.hudiSourceConfig(
HudiSourceConfig.builder()
.partitionFieldSpecConfig(onetablePartitionConfig)
.build())
.syncMode(syncMode)
.build();

table.insertRows(100);
oneTableClient.sync(perTableConfig, hudiSourceClientProvider);
checkDatasetEquivalence(
TableFormat.HUDI, table, Arrays.asList(TableFormat.ICEBERG, TableFormat.DELTA), 700);

table.insertRows(100);
oneTableClient.sync(perTableConfig, hudiSourceClientProvider);
checkDatasetEquivalence(
TableFormat.HUDI, table, Arrays.asList(TableFormat.ICEBERG, TableFormat.DELTA), 800);

table.upsertRows(insertRecords.subList(0, 20));
oneTableClient.sync(perTableConfig, hudiSourceClientProvider);
checkDatasetEquivalence(
TableFormat.HUDI, table, Arrays.asList(TableFormat.ICEBERG, TableFormat.DELTA), 800);
}
}

private Map<String, String> getTimeTravelOption(TableFormat tableFormat, Instant time) {
Map<String, String> options = new HashMap<>();
switch (tableFormat) {
Expand Down
2 changes: 1 addition & 1 deletion core/src/test/java/io/onetable/TestAbstractHudiTable.java
Original file line number Diff line number Diff line change
Expand Up @@ -761,6 +761,6 @@ public String getFilterQuery() {

@Override
public String getOrderByColumn() {
return "_hoodie_record_key";
return RECORD_KEY_FIELD_NAME;
}
}
70 changes: 70 additions & 0 deletions core/src/test/java/io/onetable/TestCommonTableHelper.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* 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 io.onetable;

import java.io.IOException;
import java.io.InputStream;
import java.io.UncheckedIOException;

import org.apache.avro.Schema;
import org.apache.spark.sql.types.StructType;

import io.onetable.avro.AvroSchemaConverter;
import io.onetable.delta.DeltaSchemaExtractor;
import io.onetable.iceberg.IcebergSchemaExtractor;
import io.onetable.model.schema.OneSchema;

public class TestCommonTableHelper {
private static Schema COMMON_SCHEMA;
private static OneSchema COMMON_ONE_SCHEMA;

static {
try (InputStream inputStream =
GenericTable.class.getClassLoader().getResourceAsStream("schemas/common_schema.avsc")) {
COMMON_SCHEMA = new Schema.Parser().parse(inputStream);
COMMON_ONE_SCHEMA = AvroSchemaConverter.getInstance().toOneSchema(COMMON_SCHEMA);
} catch (IOException ex) {
throw new UncheckedIOException(ex);
}
}

public static Schema getCommonSchemaInAvroFormat() {
return COMMON_SCHEMA;
}

public static StructType getCommonSchemaInStructFormat() {
return DeltaSchemaExtractor.getInstance().fromOneSchema(COMMON_ONE_SCHEMA);
}

public static org.apache.iceberg.Schema getCommonSchemaInIcebergFormat() {
return IcebergSchemaExtractor.getInstance().toIceberg(COMMON_ONE_SCHEMA);
}

public static String getHudiPartitionConfig() {
return "level:SIMPLE";
}

public static String getDeltaPartitionField() {
return "level";
}

public static String getIcebergPartitionField() {
return "level";
}
}
28 changes: 22 additions & 6 deletions core/src/test/java/io/onetable/TestIcebergTable.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package io.onetable;

import static io.onetable.iceberg.TestIcebergDataHelper.DEFAULT_RECORD_KEY_FIELD;
import static io.onetable.iceberg.TestIcebergDataHelper.createIcebergDataHelper;
import static org.apache.iceberg.SnapshotSummary.TOTAL_RECORDS_PROP;
import static org.junit.jupiter.api.Assertions.*;
Expand All @@ -31,6 +32,7 @@
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
Expand Down Expand Up @@ -70,7 +72,6 @@

@Getter
public class TestIcebergTable implements GenericTable<Record, String> {
private static final String DEFAULT_RECORD_KEY_FIELD = "id";
private static final List<String> DEFAULT_PARTITION_FIELDS = Collections.singletonList("level");

private final String tableName;
Expand All @@ -87,7 +88,7 @@ public static TestIcebergTable forStandardSchemaAndPartitioning(
tableName,
tempDir,
hadoopConf,
DEFAULT_RECORD_KEY_FIELD,
Optional.empty(),
Collections.singletonList(partitionField),
false);
}
Expand All @@ -98,23 +99,38 @@ public static TestIcebergTable forSchemaWithAdditionalColumnsAndPartitioning(
tableName,
tempDir,
hadoopConf,
DEFAULT_RECORD_KEY_FIELD,
Optional.empty(),
Collections.singletonList(partitionField),
true);
}

public static TestIcebergTable forGivenSchemaAndPartitioning(
String tableName,
Path tempDir,
Configuration hadoopConf,
Schema tableSchema,
String partitionField) {
return new TestIcebergTable(
tableName,
tempDir,
hadoopConf,
Optional.of(tableSchema),
Collections.singletonList(partitionField),
false);
}

public TestIcebergTable(
String tableName,
Path tempDir,
Configuration hadoopConf,
String recordKeyField,
Optional<Schema> tableSchema,
List<String> partitionFields,
boolean includeAdditionalColumns) {
this.tableName = tableName;
this.basePath = tempDir.toUri().toString();
this.icebergDataHelper =
createIcebergDataHelper(
recordKeyField, filterNullFields(partitionFields), includeAdditionalColumns);
tableSchema, filterNullFields(partitionFields), includeAdditionalColumns);
this.schema = icebergDataHelper.getTableSchema();

PartitionSpec partitionSpec = icebergDataHelper.getPartitionSpec();
Expand Down Expand Up @@ -294,7 +310,7 @@ public List<String> getColumnsToSelect() {

@Override
public String getFilterQuery() {
return String.format("%s > 'aaa'", icebergDataHelper.getRecordKeyField());
return String.format("%s > 'aaa'", DEFAULT_RECORD_KEY_FIELD);
}

public Long getLastCommitTimestamp() {
Expand Down
Loading
Loading