IcebergInputSource : Add option to toggle case sensitivity while reading columns from iceberg catalog (#16496)

* Toggle case sensitivity while reading columns from iceberg

* Fix tests

* Drop case check and set unconditionally
This commit is contained in:
Atul Mohan 2024-05-31 10:18:52 -07:00 committed by GitHub
parent 0936798122
commit b53d75758f
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
7 changed files with 80 additions and 18 deletions

View File

@ -1061,20 +1061,22 @@ The catalog object supports `local` and `hive` catalog types.
The following table lists the properties of a `local` catalog: The following table lists the properties of a `local` catalog:
|Property|Description|Required| |Property|Description|Default|Required|
|--------|-----------|---------| |--------|-----------|-------|---------|
|type|Set this value to `local`.|yes| |type|Set this value to `local`.|None|yes|
|warehousePath|The location of the warehouse associated with the catalog|yes| |warehousePath|The location of the warehouse associated with the catalog.|None|yes|
|catalogProperties|Map of any additional properties that needs to be attached to the catalog|no| |catalogProperties|Map of any additional properties that needs to be attached to the catalog.|None|no|
|caseSensitive|Toggle case sensitivity for column names during Iceberg table reads.|true|no|
The following table lists the properties of a `hive` catalog: The following table lists the properties of a `hive` catalog:
|Property|Description|Required| |Property|Description|Default|Required|
|--------|-----------|---------| |--------|-----------|-------|---------|
|type|Set this value to `hive`.|yes| |type|Set this value to `hive`.|None|yes|
|warehousePath|The location of the warehouse associated with the catalog|yes| |warehousePath|The location of the warehouse associated with the catalog.|None|yes|
|catalogUri|The URI associated with the hive catalog|yes| |catalogUri|The URI associated with the hive catalog.|None|yes|
|catalogProperties|Map of any additional properties that needs to be attached to the catalog|no| |catalogProperties|Map of any additional properties that needs to be attached to the catalog.|None|no|
|caseSensitive|Toggle case sensitivity for column names during Iceberg table reads.|true|no|
### Iceberg filter object ### Iceberg filter object

View File

@ -57,6 +57,9 @@ public class HiveIcebergCatalog extends IcebergCatalog
@JsonProperty @JsonProperty
private Map<String, String> catalogProperties; private Map<String, String> catalogProperties;
@JsonProperty
private final Boolean caseSensitive;
private final Configuration configuration; private final Configuration configuration;
private BaseMetastoreCatalog hiveCatalog; private BaseMetastoreCatalog hiveCatalog;
@ -69,6 +72,7 @@ public class HiveIcebergCatalog extends IcebergCatalog
@JsonProperty("catalogUri") String catalogUri, @JsonProperty("catalogUri") String catalogUri,
@JsonProperty("catalogProperties") @Nullable @JsonProperty("catalogProperties") @Nullable
Map<String, Object> catalogProperties, Map<String, Object> catalogProperties,
@JsonProperty("caseSensitive") Boolean caseSensitive,
@JacksonInject @Json ObjectMapper mapper, @JacksonInject @Json ObjectMapper mapper,
@JacksonInject @HiveConf Configuration configuration @JacksonInject @HiveConf Configuration configuration
) )
@ -76,6 +80,7 @@ public class HiveIcebergCatalog extends IcebergCatalog
this.warehousePath = Preconditions.checkNotNull(warehousePath, "warehousePath cannot be null"); this.warehousePath = Preconditions.checkNotNull(warehousePath, "warehousePath cannot be null");
this.catalogUri = Preconditions.checkNotNull(catalogUri, "catalogUri cannot be null"); this.catalogUri = Preconditions.checkNotNull(catalogUri, "catalogUri cannot be null");
this.catalogProperties = DynamicConfigProviderUtils.extraConfigAndSetStringMap(catalogProperties, DRUID_DYNAMIC_CONFIG_PROVIDER_KEY, mapper); this.catalogProperties = DynamicConfigProviderUtils.extraConfigAndSetStringMap(catalogProperties, DRUID_DYNAMIC_CONFIG_PROVIDER_KEY, mapper);
this.caseSensitive = caseSensitive == null ? true : caseSensitive;
this.configuration = configuration; this.configuration = configuration;
this.catalogProperties this.catalogProperties
.forEach(this.configuration::set); .forEach(this.configuration::set);
@ -137,4 +142,10 @@ public class HiveIcebergCatalog extends IcebergCatalog
{ {
return catalogProperties; return catalogProperties;
} }
@Override
public boolean isCaseSensitive()
{
return caseSensitive;
}
} }

View File

@ -50,6 +50,11 @@ public abstract class IcebergCatalog
public abstract BaseMetastoreCatalog retrieveCatalog(); public abstract BaseMetastoreCatalog retrieveCatalog();
public boolean isCaseSensitive()
{
return true;
}
/** /**
* Extract the iceberg data files upto the latest snapshot associated with the table * Extract the iceberg data files upto the latest snapshot associated with the table
* *
@ -92,6 +97,8 @@ public abstract class IcebergCatalog
if (snapshotTime != null) { if (snapshotTime != null) {
tableScan = tableScan.asOfTime(snapshotTime.getMillis()); tableScan = tableScan.asOfTime(snapshotTime.getMillis());
} }
tableScan = tableScan.caseSensitive(isCaseSensitive());
CloseableIterable<FileScanTask> tasks = tableScan.planFiles(); CloseableIterable<FileScanTask> tasks = tableScan.planFiles();
CloseableIterable.transform(tasks, FileScanTask::file) CloseableIterable.transform(tasks, FileScanTask::file)
.forEach(dataFile -> dataFilePaths.add(dataFile.path().toString())); .forEach(dataFile -> dataFilePaths.add(dataFile.path().toString()));

View File

@ -43,18 +43,23 @@ public class LocalCatalog extends IcebergCatalog
@JsonProperty @JsonProperty
private final Map<String, String> catalogProperties; private final Map<String, String> catalogProperties;
@JsonProperty
private final Boolean caseSensitive;
private BaseMetastoreCatalog catalog; private BaseMetastoreCatalog catalog;
@JsonCreator @JsonCreator
public LocalCatalog( public LocalCatalog(
@JsonProperty("warehousePath") String warehousePath, @JsonProperty("warehousePath") String warehousePath,
@JsonProperty("catalogProperties") @Nullable @JsonProperty("catalogProperties") @Nullable
Map<String, String> catalogProperties Map<String, String> catalogProperties,
@JsonProperty("caseSensitive") Boolean caseSensitive
) )
{ {
Preconditions.checkNotNull(warehousePath, "warehousePath is null"); Preconditions.checkNotNull(warehousePath, "warehousePath is null");
this.warehousePath = warehousePath; this.warehousePath = warehousePath;
this.catalogProperties = catalogProperties; this.catalogProperties = catalogProperties;
this.caseSensitive = caseSensitive == null ? true : caseSensitive;
this.catalog = retrieveCatalog(); this.catalog = retrieveCatalog();
} }
@ -71,6 +76,12 @@ public class LocalCatalog extends IcebergCatalog
return catalogProperties; return catalogProperties;
} }
@Override
public boolean isCaseSensitive()
{
return caseSensitive;
}
@Override @Override
public BaseMetastoreCatalog retrieveCatalog() public BaseMetastoreCatalog retrieveCatalog()
{ {
@ -100,12 +111,13 @@ public class LocalCatalog extends IcebergCatalog
} }
LocalCatalog that = (LocalCatalog) o; LocalCatalog that = (LocalCatalog) o;
return warehousePath.equals(that.warehousePath) return warehousePath.equals(that.warehousePath)
&& Objects.equals(catalogProperties, that.catalogProperties); && Objects.equals(catalogProperties, that.catalogProperties)
&& Objects.equals(caseSensitive, that.caseSensitive);
} }
@Override @Override
public int hashCode() public int hashCode()
{ {
return Objects.hash(warehousePath, catalogProperties); return Objects.hash(warehousePath, catalogProperties, caseSensitive);
} }
} }

View File

@ -42,6 +42,7 @@ public class HiveIcebergCatalogTest
warehouseDir.getPath(), warehouseDir.getPath(),
"hdfs://testuri", "hdfs://testuri",
new HashMap<>(), new HashMap<>(),
true,
mapper, mapper,
new Configuration() new Configuration()
); );
@ -49,6 +50,7 @@ public class HiveIcebergCatalogTest
warehouseDir.getPath(), warehouseDir.getPath(),
"hdfs://testuri", "hdfs://testuri",
null, null,
null,
mapper, mapper,
new Configuration() new Configuration()
); );
@ -68,6 +70,7 @@ public class HiveIcebergCatalogTest
warehouseDir.getPath(), warehouseDir.getPath(),
"hdfs://testuri", "hdfs://testuri",
catalogMap, catalogMap,
null,
mapper, mapper,
new Configuration() new Configuration()
); );

View File

@ -67,6 +67,7 @@ public class IcebergInputSourceTest
private IcebergCatalog testCatalog; private IcebergCatalog testCatalog;
private TableIdentifier tableIdentifier; private TableIdentifier tableIdentifier;
private File warehouseDir;
private Schema tableSchema = new Schema( private Schema tableSchema = new Schema(
Types.NestedField.required(1, "id", Types.StringType.get()), Types.NestedField.required(1, "id", Types.StringType.get()),
@ -80,8 +81,8 @@ public class IcebergInputSourceTest
@Before @Before
public void setup() throws IOException public void setup() throws IOException
{ {
final File warehouseDir = FileUtils.createTempDir(); warehouseDir = FileUtils.createTempDir();
testCatalog = new LocalCatalog(warehouseDir.getPath(), new HashMap<>()); testCatalog = new LocalCatalog(warehouseDir.getPath(), new HashMap<>(), true);
tableIdentifier = TableIdentifier.of(Namespace.of(NAMESPACE), TABLENAME); tableIdentifier = TableIdentifier.of(Namespace.of(NAMESPACE), TABLENAME);
createAndLoadTable(tableIdentifier); createAndLoadTable(tableIdentifier);
@ -187,6 +188,33 @@ public class IcebergInputSourceTest
Assert.assertEquals(1, splits.count()); Assert.assertEquals(1, splits.count());
} }
@Test
public void testCaseInsensitiveFiltering() throws IOException
{
LocalCatalog caseInsensitiveCatalog = new LocalCatalog(warehouseDir.getPath(), new HashMap<>(), false);
Table icebergTableFromSchema = testCatalog.retrieveCatalog().loadTable(tableIdentifier);
icebergTableFromSchema.updateSchema().renameColumn("name", "Name").commit();
IcebergInputSource inputSource = new IcebergInputSource(
TABLENAME,
NAMESPACE,
new IcebergEqualsFilter("name", "Foo"),
caseInsensitiveCatalog,
new LocalInputSourceFactory(),
null
);
Stream<InputSplit<List<String>>> splits = inputSource.createSplits(null, new MaxSizeSplitHintSpec(null, null));
List<File> localInputSourceList = splits.map(inputSource::withSplit)
.map(inpSource -> (LocalInputSource) inpSource)
.map(LocalInputSource::getFiles)
.flatMap(List::stream)
.collect(Collectors.toList());
Assert.assertEquals(1, inputSource.estimateNumSplits(null, new MaxSizeSplitHintSpec(1L, null)));
Assert.assertEquals(1, localInputSourceList.size());
}
@After @After
public void tearDown() public void tearDown()
{ {
@ -197,7 +225,6 @@ public class IcebergInputSourceTest
{ {
//Setup iceberg table and schema //Setup iceberg table and schema
Table icebergTableFromSchema = testCatalog.retrieveCatalog().createTable(tableIdentifier, tableSchema); Table icebergTableFromSchema = testCatalog.retrieveCatalog().createTable(tableIdentifier, tableSchema);
//Generate an iceberg record and write it to a file //Generate an iceberg record and write it to a file
GenericRecord record = GenericRecord.create(tableSchema); GenericRecord record = GenericRecord.create(tableSchema);
ImmutableList.Builder<GenericRecord> builder = ImmutableList.builder(); ImmutableList.Builder<GenericRecord> builder = ImmutableList.builder();

View File

@ -36,7 +36,7 @@ public class LocalCatalogTest
{ {
final File warehouseDir = FileUtils.createTempDir(); final File warehouseDir = FileUtils.createTempDir();
DefaultObjectMapper mapper = new DefaultObjectMapper(); DefaultObjectMapper mapper = new DefaultObjectMapper();
LocalCatalog before = new LocalCatalog(warehouseDir.getPath(), new HashMap<>()); LocalCatalog before = new LocalCatalog(warehouseDir.getPath(), new HashMap<>(), true);
LocalCatalog after = mapper.readValue( LocalCatalog after = mapper.readValue(
mapper.writeValueAsString(before), LocalCatalog.class); mapper.writeValueAsString(before), LocalCatalog.class);
Assert.assertEquals(before, after); Assert.assertEquals(before, after);