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

feat: DH-18174: Delay reading from parquet file when creating table and column location #6590

Closed
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
Original file line number Diff line number Diff line change
Expand Up @@ -87,9 +87,15 @@ public LivenessReferent asLivenessReferent() {
// TableLocationState implementation
// ------------------------------------------------------------------------------------------------------------------

protected void initializeState() {
// No-op by default, can be overridden by subclasses to initialize state on first access
}
/**
* No-op by default, can be overridden by subclasses to initialize state on first access.
* <p>
* The expectation for static locations that override this is to call {@link #handleUpdateInternal(RowSet, long)}
* instead of {@link #handleUpdate(RowSet, long)}, and {@link #handleUpdateInternal(TableLocationState)} instead of
* {@link #handleUpdate(TableLocationState)} from inside {@link #initializeState()}. Otherwise, the initialization
* logic will recurse infinitely.
*/
protected void initializeState() {}

@Override
@NotNull
Expand Down Expand Up @@ -145,6 +151,10 @@ protected final void deliverInitialSnapshot(@NotNull final Listener listener) {
*/
public final void handleUpdate(final RowSet rowSet, final long lastModifiedTimeMillis) {
initializeState();
handleUpdateInternal(rowSet, lastModifiedTimeMillis);
}

protected final void handleUpdateInternal(final RowSet rowSet, final long lastModifiedTimeMillis) {
if (state.setValues(rowSet, lastModifiedTimeMillis) && supportsSubscriptions()) {
deliverUpdateNotification();
}
Expand All @@ -158,6 +168,10 @@ public final void handleUpdate(final RowSet rowSet, final long lastModifiedTimeM
*/
public void handleUpdate(@NotNull final TableLocationState source) {
initializeState();
handleUpdateInternal(source);
}

protected final void handleUpdateInternal(@NotNull final TableLocationState source) {
if (source.copyStateValuesTo(state) && supportsSubscriptions()) {
deliverUpdateNotification();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -314,12 +314,11 @@ private void initializePages(@NotNull final ColumnDefinition<?> columnDefinition
if (pagesInitialized) {
return;
}
initializeReaders();
synchronized (this) {
if (pagesInitialized) {
return;
}

initializeReaders();
final int pageStoreCount = columnChunkReaders.length;
pageStores = new ColumnChunkPageStore[pageStoreCount];
dictionaryChunkSuppliers = new Supplier[pageStoreCount];
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,14 +67,11 @@ public class ParquetTableLocation extends AbstractTableLocation {
private Map<String, String[]> parquetColumnNameToPath;

private TableInfo tableInfo;

// Following are initialized on first access, so should only be accessed via their getters
private Map<String, GroupingColumnInfo> groupingColumns;
private Map<String, ColumnTypeInfo> columnTypes;
private List<SortColumn> sortingColumns;
private String version;

private volatile RowGroupReader[] rowGroupReaders;
private RowGroupReader[] rowGroupReaders;
// -----------------------------------------------------------------------

public ParquetTableLocation(@NotNull final TableKey tableKey,
Expand Down Expand Up @@ -128,15 +125,23 @@ private void initialize() {
tableInfo = ParquetSchemaReader
.parseMetadata(parquetMetadata.getFileMetaData().getKeyValueMetaData())
.orElse(TableInfo.builder().build());
groupingColumns = tableInfo.groupingColumnMap();
columnTypes = tableInfo.columnTypeMap();
sortingColumns = SortColumnInfo.sortColumns(tableInfo.sortingColumns());

isInitialized = true;
rowGroupReaders = IntStream.of(rowGroupIndices)
.mapToObj(idx -> parquetFileReader.getRowGroup(idx, tableInfo.version()))
.sorted(Comparator.comparingInt(rgr -> rgr.getRowGroup().getOrdinal()))
.toArray(RowGroupReader[]::new);

if (!FILE_URI_SCHEME.equals(tableLocationKey.getURI().getScheme())) {
// We do not have the last modified time for non-file URIs
handleUpdate(computeIndex(rowGroups), TableLocationState.NULL_TIME);
handleUpdateInternal(computeIndex(rowGroups), TableLocationState.NULL_TIME);
} else {
handleUpdate(computeIndex(rowGroups), new File(tableLocationKey.getURI()).lastModified());
handleUpdateInternal(computeIndex(rowGroups), new File(tableLocationKey.getURI()).lastModified());
}

isInitialized = true;
}
}

Expand All @@ -163,58 +168,21 @@ RegionedPageStore.Parameters getRegionParameters() {

public Map<String, ColumnTypeInfo> getColumnTypes() {
initialize();
if (columnTypes == null) {
columnTypes = Collections.unmodifiableMap(tableInfo.columnTypeMap());
}
return columnTypes;
}

RowGroupReader[] getRowGroupReaders() {
RowGroupReader[] local;
if ((local = rowGroupReaders) != null) {
return local;
}
synchronized (this) {
if ((local = rowGroupReaders) != null) {
return local;
}
initialize();
local = IntStream.of(rowGroupIndices)
.mapToObj(idx -> parquetFileReader.getRowGroup(idx, getVersion()))
.sorted(Comparator.comparingInt(rgr -> rgr.getRowGroup().getOrdinal()))
.toArray(RowGroupReader[]::new);
rowGroupReaders = local;
return local;
}
initialize();
return rowGroupReaders;
}

@Override
@NotNull
public List<SortColumn> getSortedColumns() {
initialize();
if (sortingColumns == null) {
sortingColumns = SortColumnInfo.sortColumns(tableInfo.sortingColumns());
}
return sortingColumns;
}

@NotNull
private Map<String, GroupingColumnInfo> getGroupingColumns() {
initialize();
if (groupingColumns == null) {
groupingColumns = Collections.unmodifiableMap(tableInfo.groupingColumnMap());
}
return groupingColumns;
}

private String getVersion() {
initialize();
if (version == null) {
version = tableInfo.version();
}
return version;
}

@NotNull
Map<String, String[]> getParquetColumnNameToPath() {
initialize();
Expand Down Expand Up @@ -254,7 +222,7 @@ private RowSet computeIndex(@NotNull final RowGroup[] rowGroups) {
public List<String[]> getDataIndexColumns() {
initialize();
final List<DataIndexInfo> dataIndexes = tableInfo.dataIndexes();
final Map<String, GroupingColumnInfo> localGroupingColumns = getGroupingColumns();
final Map<String, GroupingColumnInfo> localGroupingColumns = groupingColumns;
if (dataIndexes.isEmpty() && localGroupingColumns.isEmpty()) {
return List.of();
}
Expand All @@ -270,7 +238,7 @@ public List<String[]> getDataIndexColumns() {
public boolean hasDataIndex(@NotNull final String... columns) {
initialize();
// Check if the column name matches any of the grouping columns
if (columns.length == 1 && getGroupingColumns().containsKey(columns[0])) {
if (columns.length == 1 && groupingColumns.containsKey(columns[0])) {
// Validate the index file exists (without loading and parsing it)
final IndexFileMetadata metadata = getIndexFileMetadata(getParquetKey().getURI(), columns);
return metadata != null && parquetFileExists(metadata.fileURI);
Expand Down Expand Up @@ -346,7 +314,7 @@ private IndexFileMetadata getIndexFileMetadata(
@NotNull final String... keyColumnNames) {
if (keyColumnNames.length == 1) {
// If there's only one key column, there might be (legacy) grouping info
final GroupingColumnInfo groupingColumnInfo = getGroupingColumns().get(keyColumnNames[0]);
final GroupingColumnInfo groupingColumnInfo = groupingColumns.get(keyColumnNames[0]);
if (groupingColumnInfo != null) {
return new IndexFileMetadata(
makeRelativeURI(parentFileURI, groupingColumnInfo.groupingTablePath()),
Expand Down