/
PrimitiveDataType.java
579 lines (527 loc) · 20.1 KB
/
PrimitiveDataType.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
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
/*
* 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.datatypes;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.charset.Charset;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import org.apache.carbondata.common.logging.LogServiceFactory;
import org.apache.carbondata.core.cache.Cache;
import org.apache.carbondata.core.cache.CacheProvider;
import org.apache.carbondata.core.cache.CacheType;
import org.apache.carbondata.core.cache.dictionary.Dictionary;
import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
import org.apache.carbondata.core.constants.CarbonCommonConstants;
import org.apache.carbondata.core.datastore.ColumnType;
import org.apache.carbondata.core.datastore.row.ComplexColumnInfo;
import org.apache.carbondata.core.devapi.BiDictionary;
import org.apache.carbondata.core.devapi.DictionaryGenerationException;
import org.apache.carbondata.core.dictionary.client.DictionaryClient;
import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
import org.apache.carbondata.core.keygenerator.KeyGenException;
import org.apache.carbondata.core.keygenerator.KeyGenerator;
import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
import org.apache.carbondata.core.metadata.datatype.DataType;
import org.apache.carbondata.core.metadata.datatype.DataTypes;
import org.apache.carbondata.core.metadata.encoder.Encoding;
import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
import org.apache.carbondata.core.util.ByteUtil;
import org.apache.carbondata.core.util.CarbonUtil;
import org.apache.carbondata.core.util.DataTypeUtil;
import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
import org.apache.carbondata.processing.loading.dictionary.DictionaryServerClientDictionary;
import org.apache.carbondata.processing.loading.dictionary.DirectDictionary;
import org.apache.carbondata.processing.loading.dictionary.PreCreatedDictionary;
import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
import org.apache.log4j.Logger;
/**
* Primitive DataType stateless object used in data loading
*/
public class PrimitiveDataType implements GenericDataType<Object> {
private static final Logger LOGGER =
LogServiceFactory.getLogService(PrimitiveDataType.class.getName());
/**
* surrogate index
*/
private int index;
/**
* column name
*/
private String name;
/**
* column parent name
*/
private String parentName;
/**
* column unique id
*/
private String columnId;
/**
* key size
*/
private int keySize;
/**
* array index
*/
private int outputArrayIndex;
/**
* data counter
*/
private int dataCounter;
private BiDictionary<Integer, Object> dictionaryGenerator;
private CarbonDimension carbonDimension;
private boolean isDictionary;
private String nullFormat;
private boolean isDirectDictionary;
private DataType dataType;
private PrimitiveDataType(int outputArrayIndex, int dataCounter) {
this.outputArrayIndex = outputArrayIndex;
this.dataCounter = dataCounter;
}
/**
* constructor
*
* @param name
* @param parentName
* @param columnId
* @param isDictionary
*/
public PrimitiveDataType(String name, DataType dataType, String parentName, String columnId,
boolean isDictionary, String nullFormat) {
this.name = name;
this.parentName = parentName;
this.columnId = columnId;
this.isDictionary = isDictionary;
this.nullFormat = nullFormat;
this.dataType = dataType;
}
/**
* Constructor
* @param carbonColumn
* @param parentName
* @param columnId
* @param carbonDimension
* @param absoluteTableIdentifier
* @param client
* @param useOnePass
* @param localCache
* @param nullFormat
*/
public PrimitiveDataType(CarbonColumn carbonColumn, String parentName, String columnId,
CarbonDimension carbonDimension, AbsoluteTableIdentifier absoluteTableIdentifier,
DictionaryClient client, Boolean useOnePass, Map<Object, Integer> localCache,
String nullFormat) {
this.name = carbonColumn.getColName();
this.parentName = parentName;
this.columnId = columnId;
this.carbonDimension = carbonDimension;
this.isDictionary = isDictionaryDimension(carbonDimension);
this.nullFormat = nullFormat;
this.dataType = carbonColumn.getDataType();
DictionaryColumnUniqueIdentifier identifier =
new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
carbonDimension.getColumnIdentifier(), carbonDimension.getDataType());
try {
if (carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)
|| carbonColumn.getDataType() == DataTypes.DATE) {
dictionaryGenerator = new DirectDictionary(DirectDictionaryKeyGeneratorFactory
.getDirectDictionaryGenerator(carbonDimension.getDataType(),
getDateFormat(carbonDimension)));
isDirectDictionary = true;
} else if (carbonDimension.hasEncoding(Encoding.DICTIONARY)) {
CacheProvider cacheProvider = CacheProvider.getInstance();
Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache =
cacheProvider.createCache(CacheType.REVERSE_DICTIONARY);
Dictionary dictionary = null;
if (useOnePass) {
if (CarbonUtil.isFileExistsForGivenColumn(identifier)) {
dictionary = cache.get(identifier);
}
DictionaryMessage dictionaryMessage = new DictionaryMessage();
dictionaryMessage.setColumnName(carbonDimension.getColName());
// for table initialization
dictionaryMessage
.setTableUniqueId(absoluteTableIdentifier.getCarbonTableIdentifier().getTableId());
dictionaryMessage.setData("0");
// for generate dictionary
dictionaryMessage.setType(DictionaryMessageType.DICT_GENERATION);
dictionaryGenerator = new DictionaryServerClientDictionary(dictionary, client,
dictionaryMessage, localCache);
} else {
dictionary = cache.get(identifier);
dictionaryGenerator = new PreCreatedDictionary(dictionary);
}
}
} catch (IOException e) {
throw new RuntimeException(e);
}
}
/**
* get dateformat
* @param carbonDimension
* @return
*/
private String getDateFormat(CarbonDimension carbonDimension) {
String format;
String dateFormat = null;
if (this.carbonDimension.getDataType() == DataTypes.DATE) {
dateFormat = carbonDimension.getDateFormat();
}
if (dateFormat != null && !dateFormat.trim().isEmpty()) {
format = dateFormat;
} else {
format = CarbonUtil.getFormatFromProperty(dataType);
}
return format;
}
private boolean isDictionaryDimension(CarbonDimension carbonDimension) {
if (carbonDimension.hasEncoding(Encoding.DICTIONARY)) {
return true;
} else {
return false;
}
}
/*
* primitive column will not have any child column
*/
@Override
public void addChildren(GenericDataType children) {
}
/*
* get column name
*/
@Override
public String getName() {
return name;
}
/*
* get column parent name
*/
@Override
public String getParentName() {
return parentName;
}
/*
* get column unique id
*/
@Override
public String getColumnNames() {
return columnId;
}
/*
* primitive column will not have any children
*/
@Override
public void getAllPrimitiveChildren(List<GenericDataType> primitiveChild) {
}
/*
* get surrogate index
*/
@Override
public int getSurrogateIndex() {
return index;
}
/*
* set surrogate index
*/
@Override public void setSurrogateIndex(int surrIndex) {
if (this.carbonDimension != null && !this.carbonDimension.hasEncoding(Encoding.DICTIONARY)) {
index = 0;
} else if (this.carbonDimension == null && isDictionary == false) {
index = 0;
} else {
index = surrIndex;
}
}
@Override public boolean getIsColumnDictionary() {
return isDictionary;
}
@Override public void writeByteArray(Object input, DataOutputStream dataOutputStream,
BadRecordLogHolder logHolder) throws IOException, DictionaryGenerationException {
String parsedValue =
input == null ? null : DataTypeUtil.parseValue(input.toString(), carbonDimension);
String message = logHolder.getColumnMessageMap().get(carbonDimension.getColName());
if (this.isDictionary) {
Integer surrogateKey;
if (null == parsedValue) {
surrogateKey = CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY;
if (null == message) {
message = CarbonDataProcessorUtil
.prepareFailureReason(carbonDimension.getColName(), carbonDimension.getDataType());
logHolder.getColumnMessageMap().put(carbonDimension.getColName(), message);
logHolder.setReason(message);
}
} else {
if (dictionaryGenerator instanceof DirectDictionary && input instanceof Long) {
surrogateKey = ((DirectDictionary) dictionaryGenerator).generateKey((long) input);
} else {
surrogateKey = dictionaryGenerator.getOrGenerateKey(parsedValue);
}
if (surrogateKey == CarbonCommonConstants.INVALID_SURROGATE_KEY) {
surrogateKey = CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY;
message = CarbonDataProcessorUtil
.prepareFailureReason(carbonDimension.getColName(), carbonDimension.getDataType());
logHolder.getColumnMessageMap().put(carbonDimension.getColName(), message);
logHolder.setReason(message);
}
}
dataOutputStream.writeInt(surrogateKey);
} else {
// Transform into ByteArray for No Dictionary.
// TODO have to refactor and place all the cases present in NonDictionaryFieldConverterImpl
if (null == parsedValue && this.carbonDimension.getDataType() != DataTypes.STRING) {
updateNullValue(dataOutputStream, logHolder);
} else if (null == parsedValue || parsedValue.equals(nullFormat)) {
updateNullValue(dataOutputStream, logHolder);
} else {
String dateFormat = null;
if (this.carbonDimension.getDataType() == DataTypes.DATE) {
dateFormat = carbonDimension.getDateFormat();
} else if (this.carbonDimension.getDataType() == DataTypes.TIMESTAMP) {
dateFormat = carbonDimension.getTimestampFormat();
}
try {
if (!this.carbonDimension.getUseActualData()) {
byte[] value = null;
if (isDirectDictionary) {
int surrogateKey;
if (!(input instanceof Long)) {
SimpleDateFormat parser = new SimpleDateFormat(getDateFormat(carbonDimension));
parser.parse(parsedValue);
}
// If the input is a long value then this means that logical type was provided by
// the user using AvroCarbonWriter. In this case directly generate surrogate key
// using dictionaryGenerator.
if (dictionaryGenerator instanceof DirectDictionary && input instanceof Long) {
surrogateKey = ((DirectDictionary) dictionaryGenerator).generateKey((long) input);
} else {
surrogateKey = dictionaryGenerator.getOrGenerateKey(parsedValue);
}
if (surrogateKey == CarbonCommonConstants.INVALID_SURROGATE_KEY) {
value = new byte[0];
} else {
value = ByteUtil.toXorBytes(surrogateKey);
}
} else {
// If the input is a long value then this means that logical type was provided by
// the user using AvroCarbonWriter. In this case directly generate Bytes from value.
if (this.carbonDimension.getDataType().equals(DataTypes.DATE)
|| this.carbonDimension.getDataType().equals(DataTypes.TIMESTAMP)
&& input instanceof Long) {
if (dictionaryGenerator != null) {
value = ByteUtil.toXorBytes(((DirectDictionary) dictionaryGenerator)
.generateKey((long) input));
} else {
value = ByteUtil.toXorBytes(Long.parseLong(parsedValue));
}
} else {
value = DataTypeUtil.getBytesBasedOnDataTypeForNoDictionaryColumn(parsedValue,
this.carbonDimension.getDataType(), dateFormat);
}
if (this.carbonDimension.getDataType() == DataTypes.STRING
&& value.length > CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
throw new CarbonDataLoadingException("Dataload failed, String size cannot exceed "
+ CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " bytes");
}
}
updateValueToByteStream(dataOutputStream, value);
} else {
Object value;
if (dictionaryGenerator instanceof DirectDictionary
&& input instanceof Long) {
value = ByteUtil.toXorBytes(
((DirectDictionary) dictionaryGenerator).generateKey((long) input));
} else {
value = DataTypeUtil.getDataDataTypeForNoDictionaryColumn(parsedValue,
this.carbonDimension.getDataType(), dateFormat);
}
if (this.carbonDimension.getDataType() == DataTypes.STRING
&& value.toString().length() > CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT) {
throw new CarbonDataLoadingException("Dataload failed, String size cannot exceed "
+ CarbonCommonConstants.MAX_CHARS_PER_COLUMN_DEFAULT + " bytes");
}
if (parsedValue.length() > 0) {
updateValueToByteStream(dataOutputStream,
parsedValue.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
} else {
updateNullValue(dataOutputStream, logHolder);
}
}
} catch (NumberFormatException e) {
// Update logHolder for bad record and put null in dataOutputStream.
updateNullValue(dataOutputStream, logHolder);
} catch (CarbonDataLoadingException e) {
throw e;
} catch (ParseException ex) {
updateNullValue(dataOutputStream, logHolder);
} catch (Throwable ex) {
// TODO have to implemented the Bad Records LogHolder.
// Same like NonDictionaryFieldConverterImpl.
throw ex;
}
}
}
}
private void updateValueToByteStream(DataOutputStream dataOutputStream, byte[] value)
throws IOException {
dataOutputStream.writeShort(value.length);
dataOutputStream.write(value);
}
private void updateNullValue(DataOutputStream dataOutputStream, BadRecordLogHolder logHolder)
throws IOException {
if (this.carbonDimension.getDataType() == DataTypes.STRING) {
dataOutputStream.writeShort(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY.length);
dataOutputStream.write(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
} else {
dataOutputStream.writeShort(CarbonCommonConstants.EMPTY_BYTE_ARRAY.length);
dataOutputStream.write(CarbonCommonConstants.EMPTY_BYTE_ARRAY);
}
String message = logHolder.getColumnMessageMap().get(carbonDimension.getColName());
if (null == message) {
message = CarbonDataProcessorUtil
.prepareFailureReason(carbonDimension.getColName(), carbonDimension.getDataType());
logHolder.getColumnMessageMap().put(carbonDimension.getColName(), message);
logHolder.setReason(message);
}
}
@Override public void fillCardinality(List<Integer> dimCardWithComplex) {
if (!this.carbonDimension.hasEncoding(Encoding.DICTIONARY)) {
return;
}
dimCardWithComplex.add(dictionaryGenerator.size());
}
@Override
public void parseComplexValue(ByteBuffer byteArrayInput, DataOutputStream dataOutputStream,
KeyGenerator[] generator)
throws IOException, KeyGenException {
if (!this.isDictionary) {
int sizeOfData = byteArrayInput.getShort();
dataOutputStream.writeShort(sizeOfData);
byte[] bb = new byte[sizeOfData];
byteArrayInput.get(bb, 0, sizeOfData);
dataOutputStream.write(bb);
} else {
int data = byteArrayInput.getInt();
byte[] v = generator[index].generateKey(new int[] { data });
dataOutputStream.write(v);
}
}
/*
* get all columns count
*/
@Override
public int getColsCount() {
return 1;
}
/*
* set outputarray
*/
@Override
public void setOutputArrayIndex(int outputArrayIndex) {
this.outputArrayIndex = outputArrayIndex;
}
/*
* get output array
*/
@Override
public int getMaxOutputArrayIndex() {
return outputArrayIndex;
}
/*
* split column and return metadata and primitive column
*/
@Override public void getColumnarDataForComplexType(List<ArrayList<byte[]>> columnsArray,
ByteBuffer inputArray) {
if (!isDictionary) {
byte[] key = new byte[inputArray.getShort()];
inputArray.get(key);
columnsArray.get(outputArrayIndex).add(key);
} else {
byte[] key = new byte[keySize];
inputArray.get(key);
columnsArray.get(outputArrayIndex).add(key);
}
dataCounter++;
}
/*
* return datacounter
*/
@Override
public int getDataCounter() {
return this.dataCounter;
}
/**
* set key size
* @param keySize
*/
public void setKeySize(int keySize) {
this.keySize = keySize;
}
/*
* fill agg key block
*/
@Override
public void fillAggKeyBlock(List<Boolean> aggKeyBlockWithComplex, boolean[] aggKeyBlock) {
aggKeyBlockWithComplex.add(aggKeyBlock[index]);
}
/*
* fill block key size
*/
@Override
public void fillBlockKeySize(List<Integer> blockKeySizeWithComplex, int[] primitiveBlockKeySize) {
blockKeySizeWithComplex.add(primitiveBlockKeySize[index]);
this.keySize = primitiveBlockKeySize[index];
}
/*
* fill cardinality
*/
@Override
public void fillCardinalityAfterDataLoad(List<Integer> dimCardWithComplex,
int[] maxSurrogateKeyArray) {
dimCardWithComplex.add(maxSurrogateKeyArray[index]);
}
@Override
public GenericDataType<Object> deepCopy() {
PrimitiveDataType dataType = new PrimitiveDataType(this.outputArrayIndex, 0);
dataType.carbonDimension = this.carbonDimension;
dataType.isDictionary = this.isDictionary;
dataType.parentName = this.parentName;
dataType.columnId = this.columnId;
dataType.dictionaryGenerator = this.dictionaryGenerator;
dataType.nullFormat = this.nullFormat;
dataType.setKeySize(this.keySize);
dataType.setSurrogateIndex(this.index);
dataType.name = this.name;
dataType.dataType = this.dataType;
return dataType;
}
@Override
public void getComplexColumnInfo(List<ComplexColumnInfo> columnInfoList) {
columnInfoList.add(
new ComplexColumnInfo(ColumnType.COMPLEX_PRIMITIVE, dataType,
name, !isDictionary));
}
}