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

Ensure putSnapshot path honoring case insensitive contract #85

Merged
merged 11 commits into from
May 1, 2024
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
package com.linkedin.openhouse.catalog.e2e;

import static org.junit.jupiter.api.Assertions.*;

import com.linkedin.openhouse.tablestest.OpenHouseSparkITest;
import org.apache.spark.sql.SparkSession;
import org.junit.jupiter.api.Test;

public class CatalogOperationTest extends OpenHouseSparkITest {
@Test
public void testCasingWithCTAS() throws Exception {
autumnust marked this conversation as resolved.
Show resolved Hide resolved
try (SparkSession spark = getSparkSession()) {
// creating a casing preserving table using backtick
spark.sql("CREATE TABLE openhouse.d1.`tT1` (name string)");
spark.sql("INSERT INTO openhouse.d1.`tT1` VALUES ('foo')");

// Verifying by querying with all lower-cased name
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");
assertEquals(1, spark.sql("SELECT * FROM openhouse.d1.t2").collectAsList().size());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@
public class TableUUIDGenerator {
// TODO: r/w of tableProperties being managed in single place.
private static final String OPENHOUSE_NAMESPACE = "openhouse.";
private static final String DB_RAW_KEY = "databaseId";
private static final String TBL_RAW_KEY = "tableId";

@Autowired FsStorageProvider fsStorageProvider;

Expand Down Expand Up @@ -64,12 +66,43 @@ public UUID generateUUID(CreateUpdateTableRequestBody createUpdateTableRequestBo
public UUID generateUUID(IcebergSnapshotsRequestBody icebergSnapshotsRequestBody) {
return extractUUIDFromSnapshotJson(
icebergSnapshotsRequestBody.getJsonSnapshots(),
icebergSnapshotsRequestBody.getCreateUpdateTableRequestBody().getDatabaseId(),
icebergSnapshotsRequestBody.getCreateUpdateTableRequestBody().getTableId())
extractFromTblPropsIfExists(
autumnust marked this conversation as resolved.
Show resolved Hide resolved
getTableURI(icebergSnapshotsRequestBody),
icebergSnapshotsRequestBody.getCreateUpdateTableRequestBody().getTableProperties(),
DB_RAW_KEY),
extractFromTblPropsIfExists(
getTableURI(icebergSnapshotsRequestBody),
icebergSnapshotsRequestBody.getCreateUpdateTableRequestBody().getTableProperties(),
TBL_RAW_KEY))
.orElseGet(
() -> generateUUID(icebergSnapshotsRequestBody.getCreateUpdateTableRequestBody()));
}

/** Simple helper method to obtain tableURI from requestBody. */
private String getTableURI(IcebergSnapshotsRequestBody icebergSnapshotsRequestBody) {
autumnust marked this conversation as resolved.
Show resolved Hide resolved
return icebergSnapshotsRequestBody.getCreateUpdateTableRequestBody().getDatabaseId()
+ "."
+ icebergSnapshotsRequestBody.getCreateUpdateTableRequestBody().getTableId();
}

/**
* Extracting the value of given key from the table properties map. The main use cases are for
* tableId and databaseId where the value captured in tblproperties preserved the casing from
* creation. This casing is critical if r/w for this table occurs in a platform with different
* casing-preservation contract.
*/
private String extractFromTblPropsIfExists(
String tableURI, Map<String, String> tblProps, String rawKey) {
if (!tblProps.containsKey(OPENHOUSE_NAMESPACE + rawKey)
|| tblProps.get(OPENHOUSE_NAMESPACE + rawKey) == null) {
throw new RequestValidationFailureException(
String.format(
"Provided snapshot is invalid for %s since databaseId or tableId is missing in properties",
tableURI));
}
return tblProps.get(OPENHOUSE_NAMESPACE + rawKey);
}

/**
* Helper method to extract UUID from tableProperties. A CTAS command's commit() call provides
* "openhouse.tableUUID", if snapshot was not provided, this property is used and its path is
Expand Down Expand Up @@ -111,18 +144,10 @@ private void validatePathOfProvidedRequest(
String tableUUIDProperty,
TableType tableType) {

// Using Ids from tableProperties is to ensure casing of these Ids are properly presented as
// they were when
// initially created. Ids carried in the requestBody, if sourced from query engine, may lose
// proper casing.
String dbIdFromProps = tableProperties.get(OPENHOUSE_NAMESPACE + "databaseId");
String tblIdFromProps = tableProperties.get(OPENHOUSE_NAMESPACE + "tableId");
if (dbIdFromProps == null || tblIdFromProps == null) {
throw new RequestValidationFailureException(
String.format(
"Provided snapshot is invalid for %s.%s since databaseId or tableId is missing in properties",
databaseId, tableId));
}
String dbIdFromProps =
extractFromTblPropsIfExists(databaseId + "." + tableId, tableProperties, DB_RAW_KEY);
String tblIdFromProps =
extractFromTblPropsIfExists(databaseId + "." + tableId, tableProperties, TBL_RAW_KEY);

java.nio.file.Path previousPath =
InternalRepositoryUtils.constructTablePath(
Expand All @@ -141,8 +166,10 @@ private void validatePathOfProvidedRequest(
* validated by evaluating its "manifest-list" key.
*
* @param jsonSnapshots
* @param databaseId
* @param tableId
* @param databaseId databaseId obtained from requestBody's tableProperties to ensure casing was
* correctly captured.
* @param tableId tableId obtained from requestBody's tableProperties to ensure casing was
* correctly captured.
* @return Optional.of(UUID)
*/
private Optional<UUID> extractUUIDFromSnapshotJson(
Expand Down
Original file line number Diff line number Diff line change
@@ -1,6 +1,9 @@
package com.linkedin.openhouse.tablestest;

import com.linkedin.openhouse.internal.catalog.model.HouseTable;
import com.linkedin.openhouse.internal.catalog.model.HouseTablePrimaryKey;
import com.linkedin.openhouse.internal.catalog.repository.HouseTableRepository;
import java.util.Optional;
import org.springframework.context.annotation.Primary;
import org.springframework.stereotype.Repository;

Expand All @@ -11,4 +14,13 @@
*/
@Repository
@Primary
public interface HouseTablesH2Repository extends HouseTableRepository {}
public interface HouseTablesH2Repository extends HouseTableRepository {
Optional<HouseTable> findByDatabaseIdIgnoreCaseAndTableIdIgnoreCase(
String databaseId, String tableId);

@Override
default Optional<HouseTable> findById(HouseTablePrimaryKey houseTablePrimaryKey) {
autumnust marked this conversation as resolved.
Show resolved Hide resolved
return this.findByDatabaseIdIgnoreCaseAndTableIdIgnoreCase(
houseTablePrimaryKey.getDatabaseId(), houseTablePrimaryKey.getTableId());
}
}