/
GlobalDictionaryUtil.scala
864 lines (827 loc) · 33.1 KB
/
GlobalDictionaryUtil.scala
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
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
/*
* 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.spark.util
import java.nio.charset.Charset
import java.util.regex.Pattern
import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
import scala.language.implicitConversions
import scala.util.control.Breaks.{break, breakable}
import org.apache.commons.lang3.{ArrayUtils, StringUtils}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.io.NullWritable
import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
import org.apache.hadoop.mapreduce.security.TokenCache
import org.apache.spark.{Accumulator, SparkException}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.rdd.{NewHadoopRDD, RDD}
import org.apache.spark.sql._
import org.apache.spark.util.FileUtils
import org.apache.carbondata.common.logging.LogServiceFactory
import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
import org.apache.carbondata.core.constants.CarbonCommonConstants
import org.apache.carbondata.core.datastore.impl.FileFactory
import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnIdentifier}
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.{CarbonDimension, ColumnSchema}
import org.apache.carbondata.core.reader.CarbonDictionaryReader
import org.apache.carbondata.core.service.CarbonCommonFactory
import org.apache.carbondata.core.statusmanager.SegmentStatus
import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, DataTypeUtil}
import org.apache.carbondata.core.util.path.CarbonTablePath
import org.apache.carbondata.core.writer.CarbonDictionaryWriter
import org.apache.carbondata.processing.exception.DataLoadingException
import org.apache.carbondata.processing.loading.csvinput.{CSVInputFormat, StringArrayWritable}
import org.apache.carbondata.processing.loading.exception.NoRetryException
import org.apache.carbondata.processing.loading.model.CarbonLoadModel
import org.apache.carbondata.processing.util.CarbonLoaderUtil
import org.apache.carbondata.spark.CarbonSparkFactory
import org.apache.carbondata.spark.rdd._
import org.apache.carbondata.spark.tasks.{DictionaryWriterTask, SortIndexWriterTask}
/**
* A object which provide a method to generate global dictionary from CSV files.
*/
object GlobalDictionaryUtil {
private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
/**
* The default separator to use if none is supplied to the constructor.
*/
val DEFAULT_SEPARATOR: Char = ','
/**
* The default quote character to use if none is supplied to the
* constructor.
*/
val DEFAULT_QUOTE_CHARACTER: Char = '"'
/**
* find columns which need to generate global dictionary.
*
* @param dimensions dimension list of schema
* @param headers column headers
* @param columns column list of csv file
*/
def pruneDimensions(dimensions: Array[CarbonDimension],
headers: Array[String],
columns: Array[String]): (Array[CarbonDimension], Array[String]) = {
val dimensionBuffer = new ArrayBuffer[CarbonDimension]
val columnNameBuffer = new ArrayBuffer[String]
val dimensionsWithDict = dimensions.filter(hasEncoding(_, Encoding.DICTIONARY,
Encoding.DIRECT_DICTIONARY))
dimensionsWithDict.foreach { dim =>
breakable {
headers.zipWithIndex.foreach { h =>
if (dim.getColName.equalsIgnoreCase(h._1)) {
dimensionBuffer += dim
columnNameBuffer += columns(h._2)
break
}
}
}
}
(dimensionBuffer.toArray, columnNameBuffer.toArray)
}
/**
* use this method to judge whether CarbonDimension use some encoding or not
*
* @param dimension carbonDimension
* @param encoding the coding way of dimension
* @param excludeEncoding the coding way to exclude
*/
def hasEncoding(dimension: CarbonDimension,
encoding: Encoding,
excludeEncoding: Encoding): Boolean = {
if (dimension.isComplex()) {
val children = dimension.getListOfChildDimensions
children.asScala.exists(hasEncoding(_, encoding, excludeEncoding))
} else {
dimension.hasEncoding(encoding) &&
(excludeEncoding == null || !dimension.hasEncoding(excludeEncoding))
}
}
def gatherDimensionByEncoding(carbonLoadModel: CarbonLoadModel,
dimension: CarbonDimension,
encoding: Encoding,
excludeEncoding: Encoding,
dimensionsWithEncoding: ArrayBuffer[CarbonDimension],
forPreDefDict: Boolean) {
if (dimension.isComplex) {
val children = dimension.getListOfChildDimensions.asScala
children.foreach { c =>
gatherDimensionByEncoding(carbonLoadModel, c, encoding, excludeEncoding,
dimensionsWithEncoding, forPreDefDict)
}
} else {
if (dimension.hasEncoding(encoding) &&
(excludeEncoding == null || !dimension.hasEncoding(excludeEncoding))) {
if ((forPreDefDict && carbonLoadModel.getPredefDictFilePath(dimension) != null) ||
(!forPreDefDict && carbonLoadModel.getPredefDictFilePath(dimension) == null)) {
dimensionsWithEncoding += dimension
}
}
}
}
def getPrimDimensionWithDict(carbonLoadModel: CarbonLoadModel,
dimension: CarbonDimension,
forPreDefDict: Boolean): Array[CarbonDimension] = {
val dimensionsWithDict = new ArrayBuffer[CarbonDimension]
gatherDimensionByEncoding(carbonLoadModel, dimension, Encoding.DICTIONARY,
Encoding.DIRECT_DICTIONARY,
dimensionsWithDict, forPreDefDict)
dimensionsWithDict.toArray
}
def generateParserForChildrenDimension(dim: CarbonDimension,
format: DataFormat,
mapColumnValuesWithId:
HashMap[String, HashSet[String]],
generic: GenericParser): Unit = {
val children = dim.getListOfChildDimensions.asScala
for (i <- children.indices) {
generateParserForDimension(Some(children(i)), format.cloneAndIncreaseIndex,
mapColumnValuesWithId) match {
case Some(childDim) =>
generic.addChild(childDim)
case None =>
}
}
}
def generateParserForDimension(dimension: Option[CarbonDimension],
format: DataFormat,
mapColumnValuesWithId: HashMap[String, HashSet[String]]): Option[GenericParser] = {
dimension match {
case None =>
None
case Some(dim) =>
if (DataTypes.isArrayType(dim.getDataType) || DataTypes.isMapType(dim.getDataType)) {
val arrDim = ArrayParser(dim, format)
generateParserForChildrenDimension(dim, format, mapColumnValuesWithId, arrDim)
Some(arrDim)
} else if (DataTypes.isStructType(dim.getDataType)) {
val stuDim = StructParser(dim, format)
generateParserForChildrenDimension(dim, format, mapColumnValuesWithId, stuDim)
Some(stuDim)
} else {
Some(PrimitiveParser(dim, mapColumnValuesWithId.get(dim.getColumnId)))
}
}
}
def createDataFormat(delimiters: Array[String]): DataFormat = {
if (ArrayUtils.isNotEmpty(delimiters)) {
val patterns = delimiters.map { d =>
Pattern.compile(if (d == null) {
""
} else {
CarbonUtil.delimiterConverter(d)
})
}
DataFormat(delimiters.map(CarbonUtil.delimiterConverter(_)), 0, patterns)
} else {
null
}
}
/**
* create a instance of DictionaryLoadModel
*
* @param carbonLoadModel carbon load model
* @param table CarbonTableIdentifier
* @param dimensions column list
* @param dictFolderPath path of dictionary folder
*/
def createDictionaryLoadModel(
carbonLoadModel: CarbonLoadModel,
table: CarbonTableIdentifier,
dimensions: Array[CarbonDimension],
dictFolderPath: String,
forPreDefDict: Boolean): DictionaryLoadModel = {
val primDimensionsBuffer = new ArrayBuffer[CarbonDimension]
val isComplexes = new ArrayBuffer[Boolean]
for (i <- dimensions.indices) {
val dims = getPrimDimensionWithDict(carbonLoadModel, dimensions(i), forPreDefDict)
for (j <- dims.indices) {
primDimensionsBuffer += dims(j)
isComplexes += dimensions(i).isComplex
}
}
val primDimensions = primDimensionsBuffer.map { x => x }.toArray
val dictDetail = CarbonSparkFactory.getDictionaryDetailService.
getDictionaryDetail(dictFolderPath, primDimensions, carbonLoadModel.getTablePath)
val dictFilePaths = dictDetail.dictFilePaths
val dictFileExists = dictDetail.dictFileExists
val columnIdentifier = dictDetail.columnIdentifiers
val hdfsTempLocation = CarbonProperties.getInstance.
getProperty(CarbonCommonConstants.HDFS_TEMP_LOCATION, System.getProperty("java.io.tmpdir"))
val lockType = CarbonProperties.getInstance
.getProperty(CarbonCommonConstants.LOCK_TYPE, CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS)
val zookeeperUrl = CarbonProperties.getInstance.getProperty(CarbonCommonConstants.ZOOKEEPER_URL)
val serializationNullFormat =
carbonLoadModel.getSerializationNullFormat.split(CarbonCommonConstants.COMMA, 2)(1)
// get load count
if (null == carbonLoadModel.getLoadMetadataDetails) {
carbonLoadModel.readAndSetLoadMetadataDetails()
}
val absoluteTableIdentifier = AbsoluteTableIdentifier.from(carbonLoadModel.getTablePath, table)
DictionaryLoadModel(
absoluteTableIdentifier,
dimensions,
carbonLoadModel.getTablePath,
dictFolderPath,
dictFilePaths,
dictFileExists,
isComplexes.toArray,
primDimensions,
carbonLoadModel.getDelimiters,
columnIdentifier,
carbonLoadModel.getLoadMetadataDetails.size() == 0,
hdfsTempLocation,
lockType,
zookeeperUrl,
serializationNullFormat,
carbonLoadModel.getDefaultTimestampFormat,
carbonLoadModel.getDefaultDateFormat)
}
/**
* load and prune dictionary Rdd from csv file or input dataframe
*
* @param sqlContext sqlContext
* @param carbonLoadModel carbonLoadModel
* @param inputDF input dataframe
* @param requiredCols names of dictionary column
* @param hadoopConf hadoop configuration
* @return rdd that contains only dictionary columns
*/
private def loadInputDataAsDictRdd(sqlContext: SQLContext,
carbonLoadModel: CarbonLoadModel,
inputDF: Option[DataFrame],
requiredCols: Array[String],
hadoopConf: Configuration): RDD[Row] = {
if (inputDF.isDefined) {
inputDF.get.select(requiredCols.head, requiredCols.tail : _*).rdd
} else {
CommonUtil.configureCSVInputFormat(hadoopConf, carbonLoadModel)
hadoopConf.set(FileInputFormat.INPUT_DIR, carbonLoadModel.getFactFilePath)
val headerCols = carbonLoadModel.getCsvHeaderColumns.map(_.toLowerCase)
val header2Idx = headerCols.zipWithIndex.toMap
// index of dictionary columns in header
val dictColIdx = requiredCols.map(c => header2Idx(c.toLowerCase))
val jobConf = new JobConf(hadoopConf)
SparkHadoopUtil.get.addCredentials(jobConf)
TokenCache.obtainTokensForNamenodes(jobConf.getCredentials,
Array[Path](new Path(carbonLoadModel.getFactFilePath)),
jobConf)
val dictRdd = new NewHadoopRDD[NullWritable, StringArrayWritable](
sqlContext.sparkContext,
classOf[CSVInputFormat],
classOf[NullWritable],
classOf[StringArrayWritable],
jobConf)
.setName("global dictionary")
.map[Row] { currentRow =>
val rawRow = currentRow._2.get()
val destRow = new Array[String](dictColIdx.length)
for (i <- dictColIdx.indices) {
// dictionary index in this row
val idx = dictColIdx(i)
// copy specific dictionary value from source to dest
if (idx < rawRow.length) {
System.arraycopy(rawRow, idx, destRow, i, 1)
}
}
Row.fromSeq(destRow)
}
dictRdd
}
}
/**
* check whether global dictionary have been generated successfully or not
*
* @param status checking whether the generating is successful
*/
private def checkStatus(carbonLoadModel: CarbonLoadModel,
sqlContext: SQLContext,
model: DictionaryLoadModel,
status: Array[(Int, SegmentStatus)]) = {
var result = false
val tableName = model.table.getCarbonTableIdentifier.getTableName
status.foreach { x =>
val columnName = model.primDimensions(x._1).getColName
if (SegmentStatus.LOAD_FAILURE == x._2) {
result = true
LOGGER.error(s"table:$tableName column:$columnName generate global dictionary file failed")
}
}
if (result) {
LOGGER.error("generate global dictionary files failed")
throw new Exception("Failed to generate global dictionary files")
} else {
LOGGER.info("generate global dictionary successfully")
}
}
/**
* get external columns and whose dictionary file path
*
* @param colDictFilePath external column dict file path
* @param table table identifier
* @param dimensions dimension columns
*/
private def setPredefinedColumnDictPath(carbonLoadModel: CarbonLoadModel,
colDictFilePath: String,
table: CarbonTableIdentifier,
dimensions: Array[CarbonDimension]) = {
val colFileMapArray = colDictFilePath.split(",")
for (colPathMap <- colFileMapArray) {
val colPathMapTrim = colPathMap.trim
val colNameWithPath = colPathMapTrim.split(":")
if (colNameWithPath.length == 1) {
LOGGER.error("the format of external column dictionary should be " +
"columnName:columnPath, please check")
throw new DataLoadingException("the format of predefined column dictionary" +
" should be columnName:columnPath, please check")
}
setPredefineDict(carbonLoadModel, dimensions, table, colNameWithPath(0),
FileUtils
.getPaths(CarbonUtil
.checkAndAppendHDFSUrl(colPathMapTrim.substring(colNameWithPath(0).length + 1))))
}
}
/**
* set pre defined dictionary for dimension
*
* @param dimensions all the dimensions
* @param table carbon table identifier
* @param colName user specified column name for predefined dict
* @param colDictPath column dictionary file path
* @param parentDimName parent dimenion for complex type
*/
def setPredefineDict(carbonLoadModel: CarbonLoadModel,
dimensions: Array[CarbonDimension],
table: CarbonTableIdentifier,
colName: String,
colDictPath: String,
parentDimName: String = "") {
val middleDimName = colName.split("\\.")(0)
val dimParent = parentDimName + {
colName match {
case "" => colName
case _ =>
if (parentDimName.isEmpty) {
middleDimName
} else {
"." + middleDimName
}
}
}
// judge whether the column is exists
val preDictDimensionOption = dimensions.filter(
_.getColName.equalsIgnoreCase(dimParent))
if (preDictDimensionOption.length == 0) {
LOGGER.error(s"Column $dimParent is not a key column " +
s"in ${ table.getDatabaseName }.${ table.getTableName }")
throw new DataLoadingException(s"Column $dimParent is not a key column. " +
s"Only key column can be part of dictionary " +
s"and used in COLUMNDICT option.")
}
val preDictDimension = preDictDimensionOption(0)
if (preDictDimension.isComplex) {
val children = preDictDimension.getListOfChildDimensions.asScala.toArray
// for Array, user set ArrayFiled: path, while ArrayField has a child Array.val
val currentColName = {
if (DataTypes.isArrayType(preDictDimension.getDataType)) {
if (children(0).isComplex) {
"val." + colName.substring(middleDimName.length + 1)
} else {
"val"
}
} else {
colName.substring(middleDimName.length + 1)
}
}
setPredefineDict(carbonLoadModel, children, table, currentColName,
colDictPath, dimParent)
} else {
carbonLoadModel.setPredefDictMap(preDictDimension, colDictPath)
}
}
/**
* use external dimension column to generate global dictionary
*
* @param colDictFilePath external column dict file path
* @param table table identifier
* @param dimensions dimension column
* @param carbonLoadModel carbon load model
* @param sqlContext spark sql context
* @param dictFolderPath generated global dict file path
*/
def generatePredefinedColDictionary(colDictFilePath: String,
table: CarbonTableIdentifier,
dimensions: Array[CarbonDimension],
carbonLoadModel: CarbonLoadModel,
sqlContext: SQLContext,
dictFolderPath: String): Unit = {
// set pre defined dictionary column
setPredefinedColumnDictPath(carbonLoadModel, colDictFilePath, table, dimensions)
val dictLoadModel = createDictionaryLoadModel(carbonLoadModel, table, dimensions,
dictFolderPath, forPreDefDict = true)
// new RDD to achieve distributed column dict generation
val extInputRDD = new CarbonColumnDictGenerateRDD(carbonLoadModel, dictLoadModel,
sqlContext.sparkSession, table, dimensions, dictFolderPath)
.partitionBy(new ColumnPartitioner(dictLoadModel.primDimensions.length))
val statusList = new CarbonGlobalDictionaryGenerateRDD(sqlContext.sparkSession, extInputRDD,
dictLoadModel)
.collect()
// check result status
checkStatus(carbonLoadModel, sqlContext, dictLoadModel, statusList)
}
/* generate Dimension Parsers
*
* @param model
* @param distinctValuesList
* @return dimensionParsers
*/
def createDimensionParsers(model: DictionaryLoadModel,
distinctValuesList: ArrayBuffer[(Int, HashSet[String])]): Array[GenericParser] = {
// local combine set
val dimNum = model.dimensions.length
val primDimNum = model.primDimensions.length
val columnValues = new Array[HashSet[String]](primDimNum)
val mapColumnValuesWithId = new HashMap[String, HashSet[String]]
for (i <- 0 until primDimNum) {
columnValues(i) = new HashSet[String]
distinctValuesList += ((i, columnValues(i)))
mapColumnValuesWithId.put(model.primDimensions(i).getColumnId, columnValues(i))
}
val dimensionParsers = new Array[GenericParser](dimNum)
for (j <- 0 until dimNum) {
dimensionParsers(j) = GlobalDictionaryUtil.generateParserForDimension(
Some(model.dimensions(j)),
GlobalDictionaryUtil.createDataFormat(model.delimiters),
mapColumnValuesWithId).get
}
dimensionParsers
}
/**
* parse records in dictionary file and validate record
*
* @param x
* @param accum
* @param csvFileColumns
*/
private def parseRecord(x: String, accum: Accumulator[Int],
csvFileColumns: Array[String]): (String, String) = {
val tokens = x.split("" + DEFAULT_SEPARATOR)
var columnName: String = ""
var value: String = ""
// such as "," , "", throw ex
if (tokens.isEmpty) {
LOGGER.error("Read a bad dictionary record: " + x)
accum += 1
} else if (tokens.size == 1) {
// such as "1", "jone", throw ex
if (!x.contains(",")) {
accum += 1
} else {
try {
columnName = csvFileColumns(tokens(0).toInt)
} catch {
case _: Exception =>
LOGGER.error("Read a bad dictionary record: " + x)
accum += 1
}
}
} else {
try {
columnName = csvFileColumns(tokens(0).toInt)
value = tokens(1)
} catch {
case _: Exception =>
LOGGER.error("Read a bad dictionary record: " + x)
accum += 1
}
}
(columnName, value)
}
/**
* read local dictionary and prune column
*
* @param sqlContext
* @param csvFileColumns
* @param requireColumns
* @param allDictionaryPath
* @return allDictionaryRdd
*/
private def readAllDictionaryFiles(sqlContext: SQLContext,
csvFileColumns: Array[String],
requireColumns: Array[String],
allDictionaryPath: String,
accumulator: Accumulator[Int]) = {
var allDictionaryRdd: RDD[(String, Iterable[String])] = null
try {
// read local dictionary file, and spilt (columnIndex, columnValue)
val basicRdd = sqlContext.sparkContext.textFile(allDictionaryPath)
.map(x => parseRecord(x, accumulator, csvFileColumns))
// group by column index, and filter required columns
val requireColumnsList = requireColumns.toList
allDictionaryRdd = basicRdd
.groupByKey()
.filter(x => requireColumnsList.contains(x._1))
} catch {
case ex: Exception =>
LOGGER.error("Read dictionary files failed. Caused by: " + ex.getMessage)
throw ex
}
allDictionaryRdd
}
/**
* validate local dictionary files
*
* @param allDictionaryPath
* @return (isNonempty, isDirectory)
*/
private def validateAllDictionaryPath(allDictionaryPath: String): Boolean = {
val fileType = FileFactory.getFileType(allDictionaryPath)
val filePath = new Path(allDictionaryPath)
val file = FileFactory.getCarbonFile(filePath.toString, fileType)
val parentFile = FileFactory.getCarbonFile(filePath.getParent.toString, fileType)
// filepath regex, look like "/path/*.dictionary"
if (filePath.getName.startsWith("*")) {
val dictExt = filePath.getName.substring(1)
if (parentFile.exists()) {
val listFiles = parentFile.listFiles()
if (listFiles.exists(file =>
file.getName.endsWith(dictExt) && file.getSize > 0)) {
true
} else {
LOGGER.warn("No dictionary files found or empty dictionary files! " +
"Won't generate new dictionary.")
false
}
} else {
throw new DataLoadingException(
s"The given dictionary file path is not found : $allDictionaryPath")
}
} else {
if (file.exists()) {
if (file.getSize > 0) {
true
} else {
LOGGER.warn("No dictionary files found or empty dictionary files! " +
"Won't generate new dictionary.")
false
}
} else {
throw new DataLoadingException(
s"The given dictionary file path is not found : $allDictionaryPath")
}
}
}
/**
* generate global dictionary with SQLContext and CarbonLoadModel
*
* @param sqlContext sql context
* @param carbonLoadModel carbon load model
*/
def generateGlobalDictionary(
sqlContext: SQLContext,
carbonLoadModel: CarbonLoadModel,
hadoopConf: Configuration,
dataFrame: Option[DataFrame] = None): Unit = {
try {
val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
val dictfolderPath = CarbonTablePath.getMetadataPath(carbonLoadModel.getTablePath)
// columns which need to generate global dictionary file
val dimensions = carbonTable.getDimensionByTableName(
carbonTable.getTableName).asScala.toArray
// generate global dict from pre defined column dict file
carbonLoadModel.initPredefDictMap()
val allDictionaryPath = carbonLoadModel.getAllDictPath
if (StringUtils.isEmpty(allDictionaryPath)) {
LOGGER.info("Generate global dictionary from source data files!")
// load data by using dataSource com.databricks.spark.csv
val headers = carbonLoadModel.getCsvHeaderColumns.map(_.trim)
val colDictFilePath = carbonLoadModel.getColDictFilePath
if (colDictFilePath != null) {
// generate predefined dictionary
generatePredefinedColDictionary(colDictFilePath, carbonTableIdentifier,
dimensions, carbonLoadModel, sqlContext, dictfolderPath)
}
val headerOfInputData: Array[String] = if (dataFrame.isDefined) {
dataFrame.get.columns
} else {
headers
}
if (headers.length > headerOfInputData.length && !carbonTable.isHivePartitionTable) {
val msg = "The number of columns in the file header do not match the " +
"number of columns in the data file; Either delimiter " +
"or fileheader provided is not correct"
LOGGER.error(msg)
throw new DataLoadingException(msg)
}
// use fact file to generate global dict
val (requireDimension, requireColumnNames) = pruneDimensions(dimensions,
headers, headerOfInputData)
if (requireDimension.nonEmpty) {
// select column to push down pruning
val dictRdd = loadInputDataAsDictRdd(sqlContext, carbonLoadModel, dataFrame,
requireColumnNames, hadoopConf)
val model = createDictionaryLoadModel(carbonLoadModel, carbonTableIdentifier,
requireDimension, dictfolderPath, false)
// combine distinct value in a block and partition by column
val inputRDD = new CarbonBlockDistinctValuesCombineRDD(sqlContext.sparkSession, dictRdd,
model)
.partitionBy(new ColumnPartitioner(model.primDimensions.length))
// generate global dictionary files
val statusList = new CarbonGlobalDictionaryGenerateRDD(sqlContext.sparkSession,
inputRDD, model)
.collect()
// check result status
checkStatus(carbonLoadModel, sqlContext, model, statusList)
} else {
LOGGER.info("No column found for generating global dictionary in source data files")
}
} else {
generateDictionaryFromDictionaryFiles(
sqlContext,
carbonLoadModel,
carbonTableIdentifier,
dictfolderPath,
dimensions,
allDictionaryPath)
}
} catch {
case ex: Exception =>
if (ex.getCause != null && ex.getCause.isInstanceOf[NoRetryException]) {
LOGGER.error("generate global dictionary failed", ex.getCause)
throw new Exception("generate global dictionary failed, " +
ex.getCause.getMessage)
}
ex match {
case spx: SparkException =>
LOGGER.error("generate global dictionary failed", spx)
throw new Exception("generate global dictionary failed, " +
trimErrorMessage(spx.getMessage))
case _ =>
LOGGER.error("generate global dictionary failed", ex)
throw ex
}
}
}
def generateDictionaryFromDictionaryFiles(
sqlContext: SQLContext,
carbonLoadModel: CarbonLoadModel,
carbonTableIdentifier: CarbonTableIdentifier,
dictFolderPath: String,
dimensions: Array[CarbonDimension],
allDictionaryPath: String): Unit = {
LOGGER.info("Generate global dictionary from dictionary files!")
val allDictionaryPathAppended = CarbonUtil.checkAndAppendHDFSUrl(allDictionaryPath)
val isNonempty = validateAllDictionaryPath(allDictionaryPathAppended)
if (isNonempty) {
var headers = carbonLoadModel.getCsvHeaderColumns
headers = headers.map(headerName => headerName.trim)
// prune columns according to the CSV file header, dimension columns
val (requireDimension, requireColumnNames) = pruneDimensions(dimensions, headers, headers)
if (requireDimension.nonEmpty) {
val model = createDictionaryLoadModel(carbonLoadModel, carbonTableIdentifier,
requireDimension, dictFolderPath, false)
// check if dictionary files contains bad record
val accumulator = sqlContext.sparkContext.accumulator(0)
// read local dictionary file, and group by key
val allDictionaryRdd = readAllDictionaryFiles(sqlContext, headers,
requireColumnNames, allDictionaryPathAppended, accumulator)
// read exist dictionary and combine
val inputRDD = new CarbonAllDictionaryCombineRDD(sqlContext.sparkSession,
allDictionaryRdd, model)
.partitionBy(new ColumnPartitioner(model.primDimensions.length))
// generate global dictionary files
val statusList = new CarbonGlobalDictionaryGenerateRDD(sqlContext.sparkSession, inputRDD,
model)
.collect()
// check result status
checkStatus(carbonLoadModel, sqlContext, model, statusList)
// if the dictionary contains wrong format record, throw ex
if (accumulator.value > 0) {
throw new DataLoadingException("Data Loading failure, dictionary values are " +
"not in correct format!")
}
} else {
LOGGER.info("have no column need to generate global dictionary")
}
}
}
// Get proper error message of TextParsingException
def trimErrorMessage(input: String): String = {
var errorMessage: String = null
if (input != null && input.contains("TextParsingException:")) {
if (input.split("Hint").length > 1 &&
input.split("Hint")(0).split("TextParsingException: ").length > 1) {
errorMessage = input.split("Hint")(0).split("TextParsingException: ")(1)
} else if (input.split("Parser Configuration:").length > 1) {
errorMessage = input.split("Parser Configuration:")(0)
}
} else if (input != null && input.contains("Exception:")) {
errorMessage = input.split("Exception: ")(1).split("\n")(0)
}
errorMessage
}
/**
* This method will write dictionary file, sortindex file and dictionary meta for new dictionary
* column with default value
*
* @param columnSchema
* @param absoluteTableIdentifier
* @param defaultValue
*/
def loadDefaultDictionaryValueForNewColumn(
columnSchema: ColumnSchema,
absoluteTableIdentifier: AbsoluteTableIdentifier,
defaultValue: String): Unit = {
val dictLock = CarbonLockFactory
.getCarbonLockObj(absoluteTableIdentifier,
columnSchema.getColumnUniqueId + LockUsage.LOCK)
var isDictionaryLocked = false
try {
isDictionaryLocked = dictLock.lockWithRetries()
if (isDictionaryLocked) {
LOGGER.info(s"Successfully able to get the dictionary lock for ${
columnSchema.getColumnName
}")
} else {
sys.error(s"Dictionary file ${
columnSchema.getColumnName
} is locked for updation. Please try after some time")
}
val columnIdentifier = new ColumnIdentifier(columnSchema.getColumnUniqueId,
null,
columnSchema.getDataType)
val dictionaryColumnUniqueIdentifier: DictionaryColumnUniqueIdentifier = new
DictionaryColumnUniqueIdentifier(
absoluteTableIdentifier,
columnIdentifier,
columnIdentifier.getDataType)
val parsedValue = DataTypeUtil.normalizeColumnValueForItsDataType(defaultValue, columnSchema)
val valuesBuffer = new mutable.HashSet[String]
if (null != parsedValue) {
valuesBuffer += parsedValue
}
val dictWriteTask = new DictionaryWriterTask(valuesBuffer,
dictionary = null,
dictionaryColumnUniqueIdentifier,
columnSchema,
false
)
val distinctValues = dictWriteTask.execute
LOGGER.info(s"Dictionary file writing is successful for new column ${
columnSchema.getColumnName
}")
if (distinctValues.size() > 0) {
val sortIndexWriteTask = new SortIndexWriterTask(
dictionaryColumnUniqueIdentifier,
columnSchema.getDataType,
dictionary = null,
distinctValues)
sortIndexWriteTask.execute()
}
LOGGER.info(s"SortIndex file writing is successful for new column ${
columnSchema.getColumnName
}")
// After sortIndex writing, update dictionaryMeta
dictWriteTask.updateMetaData()
LOGGER.info(s"Dictionary meta file writing is successful for new column ${
columnSchema.getColumnName
}")
} catch {
case ex: Exception =>
LOGGER.error(ex)
throw ex
} finally {
if (dictLock != null && isDictionaryLocked) {
if (dictLock.unlock()) {
LOGGER.info(s"Dictionary ${
columnSchema.getColumnName
} Unlocked Successfully.")
} else {
LOGGER.error(s"Unable to unlock Dictionary ${
columnSchema.getColumnName
}")
}
}
}
}
}