-
Notifications
You must be signed in to change notification settings - Fork 703
/
CarbonFactDataWriterImplV3.java
408 lines (377 loc) · 16.3 KB
/
CarbonFactDataWriterImplV3.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.carbondata.processing.store.writer.v3;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import org.apache.carbondata.common.logging.LogServiceFactory;
import org.apache.carbondata.core.constants.CarbonCommonConstants;
import org.apache.carbondata.core.constants.CarbonVersionConstants;
import org.apache.carbondata.core.datastore.blocklet.BlockletEncodedColumnPage;
import org.apache.carbondata.core.datastore.blocklet.EncodedBlocklet;
import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
import org.apache.carbondata.core.datastore.page.encoding.EncodedColumnPage;
import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
import org.apache.carbondata.core.metadata.blocklet.index.BlockletBTreeIndex;
import org.apache.carbondata.core.metadata.blocklet.index.BlockletMinMaxIndex;
import org.apache.carbondata.core.metadata.index.BlockIndexInfo;
import org.apache.carbondata.core.util.CarbonMetadataUtil;
import org.apache.carbondata.core.util.CarbonProperties;
import org.apache.carbondata.core.util.CarbonUtil;
import org.apache.carbondata.core.util.DataFileFooterConverterV3;
import org.apache.carbondata.format.BlockletInfo3;
import org.apache.carbondata.format.FileFooter3;
import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
import org.apache.carbondata.processing.store.TablePage;
import org.apache.carbondata.processing.store.writer.AbstractFactDataWriter;
import static org.apache.carbondata.core.constants.CarbonCommonConstants.TABLE_BLOCKLET_SIZE;
import static org.apache.carbondata.core.constants.CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB;
import static org.apache.carbondata.core.constants.CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE;
import static org.apache.carbondata.core.constants.SortScopeOptions.SortScope.NO_SORT;
import org.apache.log4j.Logger;
/**
* Below class will be used to write the data in V3 format
* <Column1 Data ChunkV3><Column1<Page1><Page2><Page3><Page4>>
* <Column2 Data ChunkV3><Column2<Page1><Page2><Page3><Page4>>
* <Column3 Data ChunkV3><Column3<Page1><Page2><Page3><Page4>>
* <Column4 Data ChunkV3><Column4<Page1><Page2><Page3><Page4>>
*/
public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter {
private static final Logger LOGGER =
LogServiceFactory.getLogService(CarbonFactDataWriterImplV3.class.getName());
/**
* persist the page data to be written in the file
*/
private BlockletDataHolder blockletDataHolder;
/**
* Threshold of blocklet size in MB
*/
private long blockletSizeThreshold;
/**
* True if this file is sorted
*/
private boolean isSorted;
public CarbonFactDataWriterImplV3(CarbonFactDataHandlerModel model) {
super(model);
String blockletSize =
model.getTableSpec().getCarbonTable().getTableInfo().getFactTable().getTableProperties()
.get(TABLE_BLOCKLET_SIZE);
if (blockletSize == null) {
blockletSize = CarbonProperties.getInstance().getProperty(
BLOCKLET_SIZE_IN_MB, BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE);
}
blockletSizeThreshold = Long.parseLong(blockletSize) << 20;
if (blockletSizeThreshold > fileSizeInBytes) {
blockletSizeThreshold = fileSizeInBytes;
LOGGER.info("Blocklet size configure for table is: " + blockletSizeThreshold);
}
blockletDataHolder = new BlockletDataHolder(fallbackExecutorService, model);
isSorted = model.getSortScope() != NO_SORT;
}
@Override
protected void writeFooterToFile() throws CarbonDataWriterException {
try {
// get the current file position
long currentPosition = currentOffsetInFile;
// get thrift file footer instance
FileFooter3 convertFileMeta = CarbonMetadataUtil
.convertFileFooterVersion3(blockletMetadata, blockletIndex, localCardinality,
thriftColumnSchemaList.size());
convertFileMeta.setIs_sort(isSorted);
String appName = CarbonProperties.getInstance()
.getProperty(CarbonCommonConstants.CARBON_WRITTEN_BY_APPNAME);
if (appName == null) {
throw new CarbonDataWriterException(
"DataLoading failed as CARBON_WRITTEN_BY_APPNAME is null");
}
convertFileMeta.putToExtra_info(CarbonCommonConstants.CARBON_WRITTEN_BY_FOOTER_INFO, appName);
convertFileMeta.putToExtra_info(CarbonCommonConstants.CARBON_WRITTEN_VERSION,
CarbonVersionConstants.CARBONDATA_VERSION);
// fill the carbon index details
fillBlockIndexInfoDetails(convertFileMeta.getNum_rows(), carbonDataFileName, currentPosition);
// write the footer
byte[] byteArray = CarbonUtil.getByteArray(convertFileMeta);
ByteBuffer buffer =
ByteBuffer.allocate(byteArray.length + CarbonCommonConstants.LONG_SIZE_IN_BYTE);
buffer.put(byteArray);
buffer.putLong(currentPosition);
buffer.flip();
currentOffsetInFile += fileChannel.write(buffer);
} catch (IOException e) {
LOGGER.error("Problem while writing the carbon file", e);
throw new CarbonDataWriterException("Problem while writing the carbon file: ", e);
}
}
/**
* Below method will be used to write one table page data, invoked by Consumer
* @param tablePage
*/
@Override public void writeTablePage(TablePage tablePage)
throws CarbonDataWriterException,IOException {
// condition for writting all the pages
if (!tablePage.isLastPage()) {
boolean isAdded = false;
// check if size more than blocklet size then write the page to file
if (blockletDataHolder.getSize() + tablePage.getEncodedTablePage().getEncodedSize()
>= blockletSizeThreshold) {
// if blocklet size exceeds threshold, write blocklet data
if (blockletDataHolder.getNumberOfPagesAdded() == 0) {
isAdded = true;
addPageData(tablePage);
}
if (LOGGER.isDebugEnabled()) {
LOGGER.debug("Number of Pages for blocklet is: " +
blockletDataHolder.getNumberOfPagesAdded() +
" :Rows Added: " + blockletDataHolder.getTotalRows());
}
// write the data
writeBlockletToFile();
}
if (!isAdded) {
addPageData(tablePage);
}
} else {
//for last blocklet check if the last page will exceed the blocklet size then write
// existing pages and then last page
if (tablePage.getPageSize() > 0) {
addPageData(tablePage);
}
if (blockletDataHolder.getNumberOfPagesAdded() > 0) {
LOGGER.info("Number of Pages for blocklet is: " + blockletDataHolder.getNumberOfPagesAdded()
+ " :Rows Added: " + blockletDataHolder.getTotalRows());
writeBlockletToFile();
}
}
}
private void addPageData(TablePage tablePage) throws IOException {
blockletDataHolder.addPage(tablePage);
if (listener != null &&
model.getDatabaseName().equalsIgnoreCase(listener.getTblIdentifier().getDatabaseName()) &&
model.getTableName().equalsIgnoreCase(listener.getTblIdentifier().getTableName())) {
if (pageId == 0) {
listener.onBlockletStart(blockletId);
}
listener.onPageAdded(blockletId, pageId++, tablePage);
}
}
/**
* Write the collect blocklet data (blockletDataHolder) to file
*/
private void writeBlockletToFile() {
// get the list of all encoded table page
EncodedBlocklet encodedBlocklet = blockletDataHolder.getEncodedBlocklet();
int numDimensions = encodedBlocklet.getNumberOfDimension();
int numMeasures = encodedBlocklet.getNumberOfMeasure();
// get data chunks for all the column
byte[][] dataChunkBytes = new byte[numDimensions + numMeasures][];
long metadataSize = fillDataChunk(encodedBlocklet, dataChunkBytes);
// calculate the total size of data to be written
long blockletSize = blockletDataHolder.getSize() + metadataSize;
// to check if data size will exceed the block size then create a new file
createNewFileIfReachThreshold(blockletSize);
// write data to file
try {
if (currentOffsetInFile == 0) {
// write the header if file is empty
writeHeaderToFile();
}
writeBlockletToFile(dataChunkBytes);
if (listener != null &&
model.getDatabaseName().equalsIgnoreCase(listener.getTblIdentifier().getDatabaseName()) &&
model.getTableName().equalsIgnoreCase(listener.getTblIdentifier().getTableName())) {
listener.onBlockletEnd(blockletId++);
}
pageId = 0;
} catch (IOException e) {
LOGGER.error("Problem while writing file", e);
throw new CarbonDataWriterException("Problem while writing file", e);
} finally {
// clear the data holder
blockletDataHolder.clear();
}
}
/**
* Fill dataChunkBytes and return total size of page metadata
*/
private long fillDataChunk(EncodedBlocklet encodedBlocklet, byte[][] dataChunkBytes) {
int size = 0;
int numDimensions = encodedBlocklet.getNumberOfDimension();
int numMeasures = encodedBlocklet.getNumberOfMeasure();
int measureStartIndex = numDimensions;
// calculate the size of data chunks
for (int i = 0; i < numDimensions; i++) {
dataChunkBytes[i] =
CarbonUtil.getByteArray(CarbonMetadataUtil.getDimensionDataChunk3(encodedBlocklet, i));
size += dataChunkBytes[i].length;
}
for (int i = 0; i < numMeasures; i++) {
dataChunkBytes[measureStartIndex] =
CarbonUtil.getByteArray(CarbonMetadataUtil.getMeasureDataChunk3(encodedBlocklet, i));
size += dataChunkBytes[measureStartIndex].length;
measureStartIndex++;
}
return size;
}
/**
* write file header
*/
private void writeHeaderToFile() throws IOException {
byte[] fileHeader = CarbonUtil.getByteArray(
CarbonMetadataUtil.getFileHeader(
true, thriftColumnSchemaList, model.getSchemaUpdatedTimeStamp()));
ByteBuffer buffer = ByteBuffer.wrap(fileHeader);
currentOffsetInFile += fileChannel.write(buffer);
}
/**
* Write one blocklet data into file
* File format:
* <Column1 Data ChunkV3><Column1<Page1><Page2><Page3><Page4>>
* <Column2 Data ChunkV3><Column2<Page1><Page2><Page3><Page4>>
* <Column3 Data ChunkV3><Column3<Page1><Page2><Page3><Page4>>
* <Column4 Data ChunkV3><Column4<Page1><Page2><Page3><Page4>>
*/
private void writeBlockletToFile(byte[][] dataChunkBytes)
throws IOException {
long offset = currentOffsetInFile;
// to maintain the offset of each data chunk in blocklet
List<Long> currentDataChunksOffset = new ArrayList<>();
// to maintain the length of each data chunk in blocklet
List<Integer> currentDataChunksLength = new ArrayList<>();
EncodedBlocklet encodedBlocklet = blockletDataHolder.getEncodedBlocklet();
int numberOfDimension = encodedBlocklet.getNumberOfDimension();
int numberOfMeasures = encodedBlocklet.getNumberOfMeasure();
ByteBuffer buffer = null;
long dimensionOffset = 0;
long measureOffset = 0;
for (int i = 0; i < numberOfDimension; i++) {
currentDataChunksOffset.add(offset);
currentDataChunksLength.add(dataChunkBytes[i].length);
buffer = ByteBuffer.wrap(dataChunkBytes[i]);
currentOffsetInFile += fileChannel.write(buffer);
offset += dataChunkBytes[i].length;
BlockletEncodedColumnPage blockletEncodedColumnPage =
encodedBlocklet.getEncodedDimensionColumnPages().get(i);
for (EncodedColumnPage dimensionPage : blockletEncodedColumnPage
.getEncodedColumnPageList()) {
buffer = dimensionPage.getEncodedData();
int bufferSize = buffer.limit();
currentOffsetInFile += fileChannel.write(buffer);
offset += bufferSize;
}
}
dimensionOffset = offset;
int dataChunkStartIndex = encodedBlocklet.getNumberOfDimension();
for (int i = 0; i < numberOfMeasures; i++) {
currentDataChunksOffset.add(offset);
currentDataChunksLength.add(dataChunkBytes[dataChunkStartIndex].length);
buffer = ByteBuffer.wrap(dataChunkBytes[dataChunkStartIndex]);
currentOffsetInFile += fileChannel.write(buffer);
offset += dataChunkBytes[dataChunkStartIndex].length;
dataChunkStartIndex++;
BlockletEncodedColumnPage blockletEncodedColumnPage =
encodedBlocklet.getEncodedMeasureColumnPages().get(i);
for (EncodedColumnPage measurePage : blockletEncodedColumnPage
.getEncodedColumnPageList()) {
buffer = measurePage.getEncodedData();
int bufferSize = buffer.limit();
currentOffsetInFile += fileChannel.write(buffer);
offset += bufferSize;
}
}
measureOffset = offset;
blockletIndex.add(
CarbonMetadataUtil.getBlockletIndex(
encodedBlocklet, model.getSegmentProperties().getMeasures()));
BlockletInfo3 blockletInfo3 =
new BlockletInfo3(encodedBlocklet.getBlockletSize(), currentDataChunksOffset,
currentDataChunksLength, dimensionOffset, measureOffset,
encodedBlocklet.getNumberOfPages());
blockletMetadata.add(blockletInfo3);
}
/**
* Below method will be used to fill the block info details
*
* @param numberOfRows number of rows in file
* @param carbonDataFileName The name of carbonData file
* @param currentPosition current offset
*/
@Override
protected void fillBlockIndexInfoDetails(long numberOfRows, String carbonDataFileName,
long currentPosition) {
int i = 0;
DataFileFooterConverterV3 converterV3 = new DataFileFooterConverterV3();
for (org.apache.carbondata.format.BlockletIndex index : blockletIndex) {
BlockletInfo3 blockletInfo3 = blockletMetadata.get(i);
BlockletInfo blockletInfo = converterV3.getBlockletInfo(blockletInfo3,
model.getSegmentProperties().getDimensions().size());
BlockletBTreeIndex bTreeIndex = new BlockletBTreeIndex(index.b_tree_index.getStart_key(),
index.b_tree_index.getEnd_key());
BlockletMinMaxIndex minMaxIndex =
new BlockletMinMaxIndex(index.getMin_max_index().getMin_values(),
index.getMin_max_index().getMax_values(),
index.getMin_max_index().getMin_max_presence());
org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex bIndex =
new org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex(bTreeIndex,
minMaxIndex);
BlockIndexInfo biInfo =
new BlockIndexInfo(numberOfRows, carbonDataFileName, currentPosition, bIndex,
blockletInfo);
blockIndexInfoList.add(biInfo);
i++;
}
}
private byte[][] toByteArray(List<ByteBuffer> buffers) {
byte[][] arrays = new byte[buffers.size()][];
for (int i = 0; i < arrays.length; i++) {
arrays[i] = buffers.get(i).array();
}
return arrays;
}
/**
* Method will be used to close the open file channel
*
* @throws CarbonDataWriterException
*/
public void closeWriter() throws CarbonDataWriterException {
CarbonDataWriterException exception = null;
try {
commitCurrentFile(true);
writeIndexFile();
} catch (Exception e) {
LOGGER.error("Problem while writing the index file", e);
exception = new CarbonDataWriterException("Problem while writing the index file", e);
} finally {
try {
closeExecutorService();
} catch (CarbonDataWriterException e) {
if (null == exception) {
exception = e;
}
}
}
if (null != exception) {
throw exception;
}
}
@Override
public void writeFooter() throws CarbonDataWriterException {
if (this.blockletMetadata.size() > 0) {
writeFooterToFile();
}
}
}