39
39
import java .math .BigInteger ;
40
40
import java .time .Instant ;
41
41
import java .util .Arrays ;
42
+ import java .util .List ;
42
43
import java .util .Optional ;
43
44
import java .util .function .Function ;
44
45
import java .util .function .LongFunction ;
@@ -58,48 +59,70 @@ final class ParquetColumnLocation<ATTR extends Values> extends AbstractColumnLoc
58
59
private static final int MAX_PAGE_CACHE_SIZE = Configuration .getInstance ()
59
60
.getIntegerForClassWithDefault (ParquetColumnLocation .class , "maxPageCacheSize" , 8192 );
60
61
62
+ private final String columnName ;
61
63
private final String parquetColumnName ;
64
+
65
+ private volatile boolean readersInitialized ;
66
+ private final Object readersLock ;
67
+
68
+ // Access to following variables must be guarded by initializeReaders()
69
+ // -----------------------------------------------------------------------
70
+ /**
71
+ * Factory object needed for deferred initialization of the remaining fields. We delay initializing this field
72
+ * itself till we need to read the column data.
73
+ */
74
+ private ColumnChunkReader [] columnChunkReaders ;
75
+
62
76
/**
63
- * Factory object needed for deferred initialization of the remaining fields. Reference serves as a barrier to
64
- * ensure visibility of the derived fields.
77
+ * Whether the column location actually exists.
65
78
*/
66
- private volatile ColumnChunkReader [] columnChunkReaders ;
79
+ private boolean exists ;
80
+ // -----------------------------------------------------------------------
67
81
68
- // We should consider moving this to column level if needed. Column-location level likely allows more parallelism.
69
- private volatile PageCache < ATTR > pageCache ;
82
+ private volatile boolean pagesInitialized ;
83
+ private final Object pagesLock ;
70
84
85
+ // Access to following variables must be guarded by initializePages()
86
+ // -----------------------------------------------------------------------
71
87
private ColumnChunkPageStore <ATTR >[] pageStores ;
72
88
private Supplier <Chunk <ATTR >>[] dictionaryChunkSuppliers ;
73
89
private ColumnChunkPageStore <DictionaryKeys >[] dictionaryKeysPageStores ;
90
+ // -----------------------------------------------------------------------
74
91
75
92
/**
76
93
* Construct a new {@link ParquetColumnLocation} for the specified {@link ParquetTableLocation} and column name.
77
94
*
78
95
* @param tableLocation The table location enclosing this column location
79
96
* @param parquetColumnName The Parquet file column name
80
- * @param columnChunkReaders The {@link ColumnChunkReader column chunk readers} for this location
81
97
*/
82
98
ParquetColumnLocation (
83
99
@ NotNull final ParquetTableLocation tableLocation ,
84
100
@ NotNull final String columnName ,
85
- @ NotNull final String parquetColumnName ,
86
- @ Nullable final ColumnChunkReader [] columnChunkReaders ) {
101
+ @ NotNull final String parquetColumnName ) {
87
102
super (tableLocation , columnName );
103
+ this .columnName = columnName ;
88
104
this .parquetColumnName = parquetColumnName ;
89
- this .columnChunkReaders = columnChunkReaders ;
105
+ this .readersInitialized = false ;
106
+ this .readersLock = new Object ();
107
+ this .pagesInitialized = false ;
108
+ this .pagesLock = new Object ();
90
109
}
91
110
92
- private PageCache <ATTR > ensurePageCache () {
93
- PageCache <ATTR > localPageCache ;
94
- if ((localPageCache = pageCache ) != null ) {
95
- return localPageCache ;
111
+ private void initializeReaders () {
112
+ if (readersInitialized ) {
113
+ return ;
96
114
}
97
-
98
- synchronized (this ) {
99
- if ((localPageCache = pageCache ) != null ) {
100
- return localPageCache ;
115
+ synchronized (readersLock ) {
116
+ if (readersInitialized ) {
117
+ return ;
101
118
}
102
- return pageCache = new PageCache <>(INITIAL_PAGE_CACHE_SIZE , MAX_PAGE_CACHE_SIZE );
119
+ final List <String > columnPath = tl ().getColumnPath (columnName , parquetColumnName );
120
+ final ColumnChunkReader [] columnChunkReaders = Arrays .stream (tl ().getRowGroupReaders ())
121
+ .map (rgr -> rgr .getColumnChunk (columnName , columnPath ))
122
+ .toArray (ColumnChunkReader []::new );
123
+ exists = Arrays .stream (columnChunkReaders ).anyMatch (ccr -> ccr != null && ccr .numRows () > 0 );
124
+ this .columnChunkReaders = exists ? columnChunkReaders : null ;
125
+ readersInitialized = true ;
103
126
}
104
127
}
105
128
@@ -114,10 +137,8 @@ public String getImplementationName() {
114
137
115
138
@ Override
116
139
public boolean exists () {
117
- // If we see a null columnChunkReaders array, either we don't exist or we are guaranteed to
118
- // see a non-null
119
- // pageStores array
120
- return columnChunkReaders != null || pageStores != null ;
140
+ initializeReaders ();
141
+ return exists ;
121
142
}
122
143
123
144
private ParquetTableLocation tl () {
@@ -258,9 +279,9 @@ private <TYPE> ColumnRegionObject<TYPE, ATTR> makeSingleColumnRegionObject(
258
279
* @return The page stores
259
280
*/
260
281
@ NotNull
261
- public ColumnChunkPageStore <ATTR >[] getPageStores (
282
+ private ColumnChunkPageStore <ATTR >[] getPageStores (
262
283
@ NotNull final ColumnDefinition <?> columnDefinition ) {
263
- fetchValues (columnDefinition );
284
+ initializePages (columnDefinition );
264
285
return pageStores ;
265
286
}
266
287
@@ -270,9 +291,9 @@ public ColumnChunkPageStore<ATTR>[] getPageStores(
270
291
* @param columnDefinition The {@link ColumnDefinition} used to lookup type information
271
292
* @return The dictionary values chunk suppliers, or null if none exist
272
293
*/
273
- public Supplier <Chunk <ATTR >>[] getDictionaryChunkSuppliers (
294
+ private Supplier <Chunk <ATTR >>[] getDictionaryChunkSuppliers (
274
295
@ NotNull final ColumnDefinition <?> columnDefinition ) {
275
- fetchValues (columnDefinition );
296
+ initializePages (columnDefinition );
276
297
return dictionaryChunkSuppliers ;
277
298
}
278
299
@@ -285,30 +306,35 @@ public Supplier<Chunk<ATTR>>[] getDictionaryChunkSuppliers(
285
306
*/
286
307
private ColumnChunkPageStore <DictionaryKeys >[] getDictionaryKeysPageStores (
287
308
@ NotNull final ColumnDefinition <?> columnDefinition ) {
288
- fetchValues (columnDefinition );
309
+ initializePages (columnDefinition );
289
310
return dictionaryKeysPageStores ;
290
311
}
291
312
292
313
@ SuppressWarnings ("unchecked" )
293
- private void fetchValues (@ NotNull final ColumnDefinition <?> columnDefinition ) {
294
- if (columnChunkReaders == null ) {
314
+ private void initializePages (@ NotNull final ColumnDefinition <?> columnDefinition ) {
315
+ if (pagesInitialized ) {
295
316
return ;
296
317
}
297
- synchronized (this ) {
298
- if (columnChunkReaders == null ) {
318
+ synchronized (pagesLock ) {
319
+ if (pagesInitialized ) {
299
320
return ;
300
321
}
301
-
322
+ initializeReaders ();
302
323
final int pageStoreCount = columnChunkReaders .length ;
303
324
pageStores = new ColumnChunkPageStore [pageStoreCount ];
304
325
dictionaryChunkSuppliers = new Supplier [pageStoreCount ];
305
326
dictionaryKeysPageStores = new ColumnChunkPageStore [pageStoreCount ];
327
+
328
+ // We should consider moving this page-cache to column level if needed.
329
+ // Column-location level likely allows more parallelism.
330
+ final PageCache <ATTR > pageCache = new PageCache <>(INITIAL_PAGE_CACHE_SIZE , MAX_PAGE_CACHE_SIZE );
331
+
306
332
for (int psi = 0 ; psi < pageStoreCount ; ++psi ) {
307
333
final ColumnChunkReader columnChunkReader = columnChunkReaders [psi ];
308
334
try {
309
335
final ColumnChunkPageStore .CreatorResult <ATTR > creatorResult =
310
336
ColumnChunkPageStore .create (
311
- ensurePageCache () ,
337
+ pageCache ,
312
338
columnChunkReader ,
313
339
tl ().getRegionParameters ().regionMask ,
314
340
makeToPage (tl ().getColumnTypes ().get (parquetColumnName ),
@@ -325,6 +351,7 @@ private void fetchValues(@NotNull final ColumnDefinition<?> columnDefinition) {
325
351
}
326
352
327
353
columnChunkReaders = null ;
354
+ pagesInitialized = true ;
328
355
}
329
356
}
330
357
0 commit comments