Skip to content

Commit

Permalink
More review comments
Browse files Browse the repository at this point in the history
  • Loading branch information
malhotrashivam committed Jan 30, 2025
1 parent 5dde245 commit 4193f59
Show file tree
Hide file tree
Showing 2 changed files with 25 additions and 25 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -56,14 +56,14 @@ public class ParquetTableLocation extends AbstractTableLocation {

private final ParquetInstructions readInstructions;

private ParquetColumnResolver resolver;

private volatile boolean isInitialized;

// Access to all the following variables must be guarded by initialize()
// -----------------------------------------------------------------------
private ParquetFileReader parquetFileReader;

private ParquetColumnResolver resolver;

private RegionedPageStore.Parameters regionParameters;
private Map<String, String[]> parquetColumnNameToPath;

Expand Down Expand Up @@ -232,14 +232,16 @@ private RowSet computeIndex(@NotNull final RowGroup[] rowGroups) {
@NotNull
public List<String[]> getDataIndexColumns() {
initialize();
final List<DataIndexInfo> dataIndexes = tableInfo.dataIndexes();
final Map<String, GroupingColumnInfo> localGroupingColumns = groupingColumns;
if (dataIndexes.isEmpty() && localGroupingColumns.isEmpty()) {
if (tableInfo.dataIndexes().isEmpty() && localGroupingColumns.isEmpty()) {
return List.of();
}
final List<String[]> dataIndexColumns = new ArrayList<>(dataIndexes.size() + localGroupingColumns.size());
final List<String[]> dataIndexColumns =
new ArrayList<>(tableInfo.dataIndexes().size() + localGroupingColumns.size());
// Add the data indexes to the list
dataIndexes.stream().map(di -> di.columns().toArray(String[]::new)).forEach(dataIndexColumns::add);
tableInfo.dataIndexes().stream()
.map(di -> di.columns().toArray(String[]::new))
.forEach(dataIndexColumns::add);
// Add grouping columns to the list
localGroupingColumns.keySet().stream().map(colName -> new String[] {colName}).forEach(dataIndexColumns::add);
return dataIndexColumns;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3669,23 +3669,20 @@ private static void verifyMakeHandleException(final Runnable throwingRunnable) {
}
}

private static void makeNewTableLocationAndVerifyNoMakeHandleException(
private static void makeNewTableLocationAndVerifyNoException(
final Consumer<ParquetTableLocation> parquetTableLocationConsumer) {
final File dest = new File(rootFile, "real.parquet");
final Table table = TableTools.emptyTable(5).update("A=(int)i", "B=(long)i", "C=(double)i");
DataIndexer.getOrCreateDataIndex(table, "A");
writeTable(table, dest.getPath());

final ParquetTableLocationKey newTableLocationKey =
new ParquetTableLocationKey(dest.toURI(), 0, null, ParquetInstructions.EMPTY);
final ParquetTableLocation newTableLocation =
new ParquetTableLocation(StandaloneTableKey.getInstance(), newTableLocationKey, EMPTY);
try {
parquetTableLocationConsumer.accept(newTableLocation);
} catch (final Exception e) {
if (e instanceof UncheckedIOException && e.getMessage().contains("makeHandle encountered exception")) {
fail("Unexpected exception: " + e);
}
}

// The following operations should not throw exceptions
parquetTableLocationConsumer.accept(newTableLocation);
dest.delete();
}

Expand Down Expand Up @@ -3718,32 +3715,33 @@ public void testTableLocationReading() {
// Verify that all the following operations will fail when the file does not exist and pass when it does
// APIs from TableLocation
verifyMakeHandleException(nonExistentTableLocation::getDataIndexColumns);
makeNewTableLocationAndVerifyNoMakeHandleException(ParquetTableLocation::getDataIndexColumns);
makeNewTableLocationAndVerifyNoException(ParquetTableLocation::getDataIndexColumns);

verifyMakeHandleException(nonExistentTableLocation::getSortedColumns);
makeNewTableLocationAndVerifyNoMakeHandleException(ParquetTableLocation::getSortedColumns);
makeNewTableLocationAndVerifyNoException(ParquetTableLocation::getSortedColumns);

verifyMakeHandleException(nonExistentTableLocation::getColumnTypes);
makeNewTableLocationAndVerifyNoMakeHandleException(ParquetTableLocation::getColumnTypes);
makeNewTableLocationAndVerifyNoException(ParquetTableLocation::getColumnTypes);

verifyMakeHandleException(nonExistentTableLocation::hasDataIndex);
makeNewTableLocationAndVerifyNoException(ParquetTableLocation::hasDataIndex);

// Assuming here there will be an index on column "A"
verifyMakeHandleException(nonExistentTableLocation::getDataIndex);
makeNewTableLocationAndVerifyNoMakeHandleException(ParquetTableLocation::getDataIndex);
makeNewTableLocationAndVerifyNoException(tableLocation -> tableLocation.getDataIndex("A"));

verifyMakeHandleException(nonExistentTableLocation::loadDataIndex);
makeNewTableLocationAndVerifyNoMakeHandleException(ParquetTableLocation::loadDataIndex);

verifyMakeHandleException(nonExistentTableLocation::hasDataIndex);
makeNewTableLocationAndVerifyNoMakeHandleException(ParquetTableLocation::hasDataIndex);
makeNewTableLocationAndVerifyNoException(tableLocation -> tableLocation.loadDataIndex("A"));

// APIs from TableLocationState
verifyMakeHandleException(nonExistentTableLocation::getRowSet);
makeNewTableLocationAndVerifyNoMakeHandleException(ParquetTableLocation::getRowSet);
makeNewTableLocationAndVerifyNoException(ParquetTableLocation::getRowSet);

verifyMakeHandleException(nonExistentTableLocation::getSize);
makeNewTableLocationAndVerifyNoMakeHandleException(ParquetTableLocation::getSize);
makeNewTableLocationAndVerifyNoException(ParquetTableLocation::getSize);

verifyMakeHandleException(nonExistentTableLocation::getLastModifiedTimeMillis);
makeNewTableLocationAndVerifyNoMakeHandleException(ParquetTableLocation::getLastModifiedTimeMillis);
makeNewTableLocationAndVerifyNoException(ParquetTableLocation::getLastModifiedTimeMillis);

verifyMakeHandleException(() -> nonExistentTableLocation.handleUpdate(new TrackingWritableRowSetImpl(), 0));

Expand Down

0 comments on commit 4193f59

Please sign in to comment.