-
Notifications
You must be signed in to change notification settings - Fork 40
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Ensure putSnapshot path honoring case insensitive contract (#85)
## Summary This is a bug fixes for put-snapshot code path, where UUID-extraction process is using `tableId` and `databaseId` from the request itself. Those ids, if directly obtained from top-level request body, can lost casing if the requests were from platform like Spark SQL. Since the underlying storage (e.g. HDFS ) is case sensitive in its path URL, we will need to ensure the original casing info when issue the first commit as part of CTAS is preserved in the process of UUID-extraction of the second commit. The other parts in this PR is to ensure, when `put` is happening, the `tableDto` is not always built from scratch when there's existing object discovered by `findById` method previously. This is done by switch `orElse` method to `orElseGet`, in which the latter will only call the supplier lazily when the calling object is absent. This leads to a wasteful implementation as well as confusion on the code stack. This PR also include: - Some refactoring to share code with existing code. - Ensure `text-fixtures` module has its repository horning case insensitive contract so that this behavior can be tested through embedded instances. - Testing the casing contract in both SQL and Catalog API.
- Loading branch information
Showing
8 changed files
with
225 additions
and
52 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
91 changes: 91 additions & 0 deletions
91
integrations/spark/openhouse-spark-runtime/src/test/java/CatalogOperationTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,91 @@ | ||
import com.linkedin.openhouse.tablestest.OpenHouseSparkITest; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.concurrent.atomic.AtomicReference; | ||
import org.apache.iceberg.CatalogUtil; | ||
import org.apache.iceberg.DataFile; | ||
import org.apache.iceberg.DataFiles; | ||
import org.apache.iceberg.PartitionSpec; | ||
import org.apache.iceberg.Schema; | ||
import org.apache.iceberg.Table; | ||
import org.apache.iceberg.catalog.Catalog; | ||
import org.apache.iceberg.catalog.TableIdentifier; | ||
import org.apache.iceberg.types.Types; | ||
import org.apache.spark.sql.SparkSession; | ||
import org.junit.jupiter.api.Assertions; | ||
import org.junit.jupiter.api.Test; | ||
import scala.collection.JavaConverters; | ||
|
||
public class CatalogOperationTest extends OpenHouseSparkITest { | ||
@Test | ||
public void testCasingWithCTAS() throws Exception { | ||
try (SparkSession spark = getSparkSession()) { | ||
// creating a casing preserving table using backtick | ||
spark.sql("CREATE TABLE openhouse.d1.`tT1` (name string)"); | ||
// testing writing behavior, note the casing of tt1 is intentionally changed. | ||
spark.sql("INSERT INTO openhouse.d1.Tt1 VALUES ('foo')"); | ||
|
||
// Verifying by querying with all lower-cased name | ||
Assertions.assertEquals( | ||
1, spark.sql("SELECT * from openhouse.d1.tt1").collectAsList().size()); | ||
// ctas but referring with lower-cased name | ||
spark.sql("CREATE TABLE openhouse.d1.t2 AS SELECT * from openhouse.d1.tt1"); | ||
Assertions.assertEquals(1, spark.sql("SELECT * FROM openhouse.d1.t2").collectAsList().size()); | ||
} | ||
} | ||
|
||
@Test | ||
public void testCatalogWriteAPI() throws Exception { | ||
try (SparkSession spark = getSparkSession()) { | ||
Catalog icebergCatalog = getOpenHouseCatalog(spark); | ||
// Create a table | ||
Schema schema = new Schema(Types.NestedField.required(1, "name", Types.StringType.get())); | ||
TableIdentifier tableIdentifier = TableIdentifier.of("db", "aaa"); | ||
icebergCatalog.createTable(tableIdentifier, schema); | ||
|
||
// Write into data with intentionally changed casing in name | ||
TableIdentifier tableIdentifierUpperTblName = TableIdentifier.of("db", "AAA"); | ||
|
||
DataFile fooDataFile = | ||
DataFiles.builder(PartitionSpec.unpartitioned()) | ||
.withPath("/path/to/data-a.parquet") | ||
.withFileSizeInBytes(10) | ||
.withRecordCount(1) | ||
.build(); | ||
AtomicReference<Table> tableRef = new AtomicReference<>(); | ||
Assertions.assertDoesNotThrow( | ||
() -> { | ||
Table loadedTable = icebergCatalog.loadTable(tableIdentifierUpperTblName); | ||
tableRef.set(loadedTable); | ||
}); | ||
Table table = tableRef.get(); | ||
Assertions.assertDoesNotThrow( | ||
() -> { | ||
table.newAppend().appendFile(fooDataFile).commit(); | ||
}); | ||
} | ||
} | ||
|
||
/** | ||
* This is a copy of com.linkedin.openhouse.jobs.spark.Operations#getCatalog() temporarily. | ||
* Refactoring these pieces require deployment coordination, thus we shall create an artifact | ||
* module that can be pulled by :apps module. | ||
*/ | ||
private Catalog getOpenHouseCatalog(SparkSession spark) { | ||
final Map<String, String> catalogProperties = new HashMap<>(); | ||
final String catalogPropertyPrefix = String.format("spark.sql.catalog.openhouse."); | ||
final Map<String, String> sparkProperties = JavaConverters.mapAsJavaMap(spark.conf().getAll()); | ||
for (Map.Entry<String, String> entry : sparkProperties.entrySet()) { | ||
if (entry.getKey().startsWith(catalogPropertyPrefix)) { | ||
catalogProperties.put( | ||
entry.getKey().substring(catalogPropertyPrefix.length()), entry.getValue()); | ||
} | ||
} | ||
// this initializes the catalog based on runtime Catalog class passed in catalog-impl conf. | ||
return CatalogUtil.loadCatalog( | ||
sparkProperties.get("spark.sql.catalog.openhouse.catalog-impl"), | ||
"openhouse", | ||
catalogProperties, | ||
spark.sparkContext().hadoopConfiguration()); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters