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 #6606

Merged
Merged
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 @@ -63,6 +63,7 @@ final class ParquetColumnLocation<ATTR extends Values> extends AbstractColumnLoc
private final String parquetColumnName;

private volatile boolean readersInitialized;
private final Object readersLock;

// Access to following variables must be guarded by initializeReaders()
// -----------------------------------------------------------------------
Expand All @@ -79,6 +80,7 @@ final class ParquetColumnLocation<ATTR extends Values> extends AbstractColumnLoc
// -----------------------------------------------------------------------

private volatile boolean pagesInitialized;
private final Object pagesLock;

// Access to following variables must be guarded by initializePages()
// -----------------------------------------------------------------------
Expand All @@ -101,14 +103,16 @@ final class ParquetColumnLocation<ATTR extends Values> extends AbstractColumnLoc
this.columnName = columnName;
this.parquetColumnName = parquetColumnName;
this.readersInitialized = false;
this.readersLock = new Object();
this.pagesInitialized = false;
this.pagesLock = new Object();
}

private void initializeReaders() {
if (readersInitialized) {
return;
}
synchronized (this) {
synchronized (readersLock) {
if (readersInitialized) {
return;
}
Expand Down Expand Up @@ -311,7 +315,7 @@ private void initializePages(@NotNull final ColumnDefinition<?> columnDefinition
if (pagesInitialized) {
return;
}
synchronized (this) {
synchronized (pagesLock) {
if (pagesInitialized) {
return;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,9 @@ public class ParquetTableLocation extends AbstractTableLocation {

// Access to all the following variables must be guarded by initialize()
// -----------------------------------------------------------------------
private ParquetFileReader parquetFileReader;
private int[] rowGroupIndices;
malhotrashivam marked this conversation as resolved.
Show resolved Hide resolved

private ParquetColumnResolver resolver;

private RegionedPageStore.Parameters regionParameters;
Expand All @@ -70,7 +73,7 @@ public class ParquetTableLocation extends AbstractTableLocation {
private Map<String, ColumnTypeInfo> columnTypes;
private List<SortColumn> sortingColumns;

private RowGroupReader[] rowGroupReaders;
private volatile RowGroupReader[] rowGroupReaders;
malhotrashivam marked this conversation as resolved.
Show resolved Hide resolved
// -----------------------------------------------------------------------

public ParquetTableLocation(@NotNull final TableKey tableKey,
Expand All @@ -89,10 +92,8 @@ private void initialize() {
if (isInitialized) {
return;
}
final ParquetFileReader parquetFileReader;
final ParquetMetadata parquetMetadata;
final ParquetTableLocationKey tableLocationKey = getParquetKey();
final int[] rowGroupIndices;
synchronized (tableLocationKey) {
// Following methods are internally synchronized, we synchronize them together here to minimize
// lock/unlock calls
Expand Down Expand Up @@ -127,11 +128,6 @@ private void initialize() {
columnTypes = tableInfo.columnTypeMap();
sortingColumns = SortColumnInfo.sortColumns(tableInfo.sortingColumns());

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
handleUpdateInternal(computeRowSet(rowGroups), TableLocationState.NULL_TIME);
Expand Down Expand Up @@ -170,8 +166,22 @@ public Map<String, ColumnTypeInfo> getColumnTypes() {
}

RowGroupReader[] getRowGroupReaders() {
initialize();
return rowGroupReaders;
RowGroupReader[] local;
if ((local = rowGroupReaders) != null) {
return local;
}
synchronized (this) {
if ((local = rowGroupReaders) != null) {
return local;
}
initialize();
rcaudy marked this conversation as resolved.
Show resolved Hide resolved
local = IntStream.of(rowGroupIndices)
.mapToObj(idx -> parquetFileReader.getRowGroup(idx, tableInfo.version()))
.sorted(Comparator.comparingInt(rgr -> rgr.getRowGroup().getOrdinal()))
.toArray(RowGroupReader[]::new);
rowGroupReaders = local;
return local;
}
}

@Override
Expand Down
Loading