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