-
Notifications
You must be signed in to change notification settings - Fork 704
/
CarbonLoadDataCommand.scala
1130 lines (1082 loc) · 47.4 KB
/
CarbonLoadDataCommand.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
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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.spark.sql.execution.command.management
import java.text.SimpleDateFormat
import java.util
import java.util.UUID
import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import org.apache.commons.lang3.StringUtils
import org.apache.hadoop.conf.Configuration
import org.apache.spark.rdd.RDD
import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, UnresolvedAttribute}
import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.expressions.{Ascending, AttributeReference, SortOrder}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Sort}
import org.apache.spark.sql.execution.LogicalRDD
import org.apache.spark.sql.execution.SQLExecution.EXECUTION_ID_KEY
import org.apache.spark.sql.execution.command.{AtomicRunnableCommand, DataLoadTableFileMapping, UpdateTableModel}
import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, FindDataSourceTable, HadoopFsRelation, LogicalRelation, SparkCarbonTableFormat}
import org.apache.spark.sql.hive.CarbonRelation
import org.apache.spark.sql.optimizer.CarbonFilters
import org.apache.spark.sql.types._
import org.apache.spark.storage.StorageLevel
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.{CarbonReflectionUtils, CausedBy, FileUtils}
import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
import org.apache.carbondata.converter.SparkDataTypeConverterImpl
import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
import org.apache.carbondata.core.datamap.DataMapStoreManager
import org.apache.carbondata.core.datastore.compression.CompressorFactory
import org.apache.carbondata.core.datastore.impl.FileFactory
import org.apache.carbondata.core.dictionary.server.{DictionaryServer, NonSecureDictionaryServer}
import org.apache.carbondata.core.dictionary.service.NonSecureDictionaryServiceProvider
import org.apache.carbondata.core.indexstore.PartitionSpec
import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, ICarbonLock, LockUsage}
import org.apache.carbondata.core.metadata.SegmentFileStore
import org.apache.carbondata.core.metadata.encoder.Encoding
import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, TupleIdEnum}
import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
import org.apache.carbondata.core.util._
import org.apache.carbondata.core.util.path.CarbonTablePath
import org.apache.carbondata.events.{BuildDataMapPostExecutionEvent, BuildDataMapPreExecutionEvent, OperationContext, OperationListenerBus}
import org.apache.carbondata.events.exception.PreEventException
import org.apache.carbondata.processing.exception.DataLoadingException
import org.apache.carbondata.processing.loading.TableProcessingOperations
import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadMetadataEvent, LoadTablePostExecutionEvent, LoadTablePreExecutionEvent}
import org.apache.carbondata.processing.loading.exception.NoRetryException
import org.apache.carbondata.processing.loading.model.{CarbonLoadModelBuilder, LoadOption}
import org.apache.carbondata.processing.loading.model.CarbonLoadModel
import org.apache.carbondata.processing.loading.sort.SortScopeOptions
import org.apache.carbondata.processing.util.{CarbonBadRecordUtil, CarbonDataProcessorUtil, CarbonLoaderUtil}
import org.apache.carbondata.spark.dictionary.provider.SecureDictionaryServiceProvider
import org.apache.carbondata.spark.dictionary.server.SecureDictionaryServer
import org.apache.carbondata.spark.load.{CsvRDDHelper, DataLoadProcessorStepOnSpark}
import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, SerializableConfiguration}
import org.apache.carbondata.spark.util.{CarbonScalaUtil, GlobalDictionaryUtil}
case class CarbonLoadDataCommand(
databaseNameOp: Option[String],
tableName: String,
factPathFromUser: String,
dimFilesPath: Seq[DataLoadTableFileMapping],
options: scala.collection.immutable.Map[String, String],
isOverwriteTable: Boolean,
var inputSqlString: String = null,
var dataFrame: Option[DataFrame] = None,
updateModel: Option[UpdateTableModel] = None,
var tableInfoOp: Option[TableInfo] = None,
var internalOptions: Map[String, String] = Map.empty,
partition: Map[String, Option[String]] = Map.empty,
logicalPlan: Option[LogicalPlan] = None,
var operationContext: OperationContext = new OperationContext) extends AtomicRunnableCommand {
var table: CarbonTable = _
var logicalPartitionRelation: LogicalRelation = _
var sizeInBytes: Long = _
var currPartitions: util.List[PartitionSpec] = _
var parentTablePath: String = _
override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
val dbName = CarbonEnv.getDatabaseName(databaseNameOp)(sparkSession)
table = if (tableInfoOp.isDefined) {
CarbonTable.buildFromTableInfo(tableInfoOp.get)
} else {
val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
.lookupRelation(Option(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
if (relation == null) {
throw new NoSuchTableException(dbName, tableName)
}
if (null == relation.carbonTable) {
LOGGER.error(s"Data loading failed. table not found: $dbName.$tableName")
LOGGER.audit(s"Data loading failed. table not found: $dbName.$tableName")
throw new NoSuchTableException(dbName, tableName)
}
relation.carbonTable
}
if (table.isHivePartitionTable) {
logicalPartitionRelation =
new FindDataSourceTable(sparkSession).apply(
sparkSession.sessionState.catalog.lookupRelation(
TableIdentifier(tableName, databaseNameOp))).collect {
case l: LogicalRelation => l
}.head
sizeInBytes = logicalPartitionRelation.relation.sizeInBytes
}
if (table.isChildDataMap) {
val parentTableIdentifier = table.getTableInfo.getParentRelationIdentifiers.get(0)
parentTablePath = CarbonEnv
.getCarbonTable(Some(parentTableIdentifier.getDatabaseName),
parentTableIdentifier.getTableName)(sparkSession).getTablePath
}
operationContext.setProperty("isOverwrite", isOverwriteTable)
if(CarbonUtil.hasAggregationDataMap(table)) {
val loadMetadataEvent = new LoadMetadataEvent(table, false)
OperationListenerBus.getInstance().fireEvent(loadMetadataEvent, operationContext)
}
Seq.empty
}
override def processData(sparkSession: SparkSession): Seq[Row] = {
val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
val carbonProperty: CarbonProperties = CarbonProperties.getInstance()
var concurrentLoadLock: Option[ICarbonLock] = None
carbonProperty.addProperty("zookeeper.enable.lock", "false")
currPartitions = if (table.isHivePartitionTable) {
CarbonFilters.getCurrentPartitions(
sparkSession,
table) match {
case Some(parts) => new util.ArrayList(parts.toList.asJava)
case _ => null
}
} else {
null
}
// get the value of 'spark.executor.cores' from spark conf, default value is 1
val sparkExecutorCores = sparkSession.sparkContext.conf.get("spark.executor.cores", "1")
// get the value of 'carbon.number.of.cores.while.loading' from carbon properties,
// default value is the value of 'spark.executor.cores'
val numCoresLoading =
try {
CarbonProperties.getInstance()
.getProperty(CarbonCommonConstants.NUM_CORES_LOADING, sparkExecutorCores)
} catch {
case exc: NumberFormatException =>
LOGGER.error("Configured value for property " + CarbonCommonConstants.NUM_CORES_LOADING
+ " is wrong. Falling back to the default value "
+ sparkExecutorCores)
sparkExecutorCores
}
// update the property with new value
carbonProperty.addProperty(CarbonCommonConstants.NUM_CORES_LOADING, numCoresLoading)
val dbName = CarbonEnv.getDatabaseName(databaseNameOp)(sparkSession)
val hadoopConf = sparkSession.sessionState.newHadoopConf()
val carbonLoadModel = new CarbonLoadModel()
try {
val tableProperties = table.getTableInfo.getFactTable.getTableProperties
val optionsFinal = LoadOption.fillOptionWithDefaultValue(options.asJava)
optionsFinal.put("sort_scope", tableProperties.asScala.getOrElse("sort_scope",
carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE,
carbonProperty.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))))
optionsFinal
.put("bad_record_path", CarbonBadRecordUtil.getBadRecordsPath(options.asJava, table))
val factPath = if (dataFrame.isDefined) {
""
} else {
FileUtils.getPaths(factPathFromUser, hadoopConf)
}
carbonLoadModel.setParentTablePath(parentTablePath)
carbonLoadModel.setFactFilePath(factPath)
carbonLoadModel.setCarbonTransactionalTable(table.getTableInfo.isTransactionalTable)
carbonLoadModel.setAggLoadRequest(
internalOptions.getOrElse(CarbonCommonConstants.IS_INTERNAL_LOAD_CALL, "false").toBoolean)
carbonLoadModel.setSegmentId(internalOptions.getOrElse("mergedSegmentName", ""))
val columnCompressor = table.getTableInfo.getFactTable.getTableProperties.asScala
.getOrElse(CarbonCommonConstants.COMPRESSOR,
CompressorFactory.getInstance().getCompressor.getName)
carbonLoadModel.setColumnCompressor(columnCompressor)
val javaPartition = mutable.Map[String, String]()
partition.foreach { case (k, v) =>
if (v.isEmpty) javaPartition(k) = null else javaPartition(k) = v.get
}
new CarbonLoadModelBuilder(table).build(
options.asJava,
optionsFinal,
carbonLoadModel,
hadoopConf,
javaPartition.asJava,
dataFrame.isDefined)
// Delete stale segment folders that are not in table status but are physically present in
// the Fact folder
LOGGER.info(s"Deleting stale folders if present for table $dbName.$tableName")
TableProcessingOperations.deletePartialLoadDataIfExist(table, false)
var isUpdateTableStatusRequired = false
// if the table is child then extract the uuid from the operation context and the parent would
// already generated UUID.
// if parent table then generate a new UUID else use empty.
val uuid = if (table.isChildDataMap) {
Option(operationContext.getProperty("uuid")).getOrElse("").toString
} else if (table.hasAggregationDataMap) {
UUID.randomUUID().toString
} else {
""
}
try {
operationContext.setProperty("uuid", uuid)
val loadTablePreExecutionEvent: LoadTablePreExecutionEvent =
new LoadTablePreExecutionEvent(
table.getCarbonTableIdentifier,
carbonLoadModel)
operationContext.setProperty("isOverwrite", isOverwriteTable)
OperationListenerBus.getInstance.fireEvent(loadTablePreExecutionEvent, operationContext)
// Add pre event listener for index datamap
val tableDataMaps = DataMapStoreManager.getInstance().getAllDataMap(table)
val dataMapOperationContext = new OperationContext()
if (null != tableDataMaps) {
val dataMapNames: mutable.Buffer[String] =
tableDataMaps.asScala.map(dataMap => dataMap.getDataMapSchema.getDataMapName)
val buildDataMapPreExecutionEvent: BuildDataMapPreExecutionEvent =
new BuildDataMapPreExecutionEvent(sparkSession,
table.getAbsoluteTableIdentifier, dataMapNames)
OperationListenerBus.getInstance().fireEvent(buildDataMapPreExecutionEvent,
dataMapOperationContext)
}
// First system has to partition the data first and then call the load data
LOGGER.info(s"Initiating Direct Load for the Table : ($dbName.$tableName)")
concurrentLoadLock = acquireConcurrentLoadLock()
// Clean up the old invalid segment data before creating a new entry for new load.
SegmentStatusManager.deleteLoadsAndUpdateMetadata(table, false, currPartitions)
// add the start entry for the new load in the table status file
if (updateModel.isEmpty && !table.isHivePartitionTable) {
CarbonLoaderUtil.readAndUpdateLoadProgressInTableMeta(
carbonLoadModel,
isOverwriteTable)
isUpdateTableStatusRequired = true
}
if (isOverwriteTable) {
LOGGER.info(s"Overwrite of carbon table with $dbName.$tableName is in progress")
}
// if table is an aggregate table then disable single pass.
if (carbonLoadModel.isAggLoadRequest) {
carbonLoadModel.setUseOnePass(false)
}
// start dictionary server when use one pass load and dimension with DICTIONARY
// encoding is present.
val allDimensions =
carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.getAllDimensions.asScala.toList
val createDictionary = allDimensions.exists {
carbonDimension => carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
!carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)
}
if (!createDictionary) {
carbonLoadModel.setUseOnePass(false)
}
// Create table and metadata folders if not exist
if (carbonLoadModel.isCarbonTransactionalTable) {
val metadataDirectoryPath = CarbonTablePath.getMetadataPath(table.getTablePath)
val fileType = FileFactory.getFileType(metadataDirectoryPath)
if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
FileFactory.mkdirs(metadataDirectoryPath, fileType)
}
} else {
carbonLoadModel.setSegmentId(System.currentTimeMillis().toString)
}
val partitionStatus = SegmentStatus.SUCCESS
val columnar = sparkSession.conf.get("carbon.is.columnar.storage", "true").toBoolean
if (carbonLoadModel.getUseOnePass) {
loadDataUsingOnePass(
sparkSession,
carbonProperty,
carbonLoadModel,
columnar,
partitionStatus,
hadoopConf,
operationContext,
LOGGER)
} else {
loadData(
sparkSession,
carbonLoadModel,
columnar,
partitionStatus,
hadoopConf,
operationContext,
LOGGER)
}
val loadTablePostExecutionEvent: LoadTablePostExecutionEvent =
new LoadTablePostExecutionEvent(
table.getCarbonTableIdentifier,
carbonLoadModel)
OperationListenerBus.getInstance.fireEvent(loadTablePostExecutionEvent, operationContext)
if (null != tableDataMaps) {
val buildDataMapPostExecutionEvent: BuildDataMapPostExecutionEvent =
BuildDataMapPostExecutionEvent(sparkSession, table.getAbsoluteTableIdentifier)
OperationListenerBus.getInstance()
.fireEvent(buildDataMapPostExecutionEvent, dataMapOperationContext)
}
} catch {
case CausedBy(ex: NoRetryException) =>
// update the load entry in table status file for changing the status to marked for delete
if (isUpdateTableStatusRequired) {
CarbonLoaderUtil.updateTableStatusForFailure(carbonLoadModel, uuid)
}
LOGGER.error(ex, s"Dataload failure for $dbName.$tableName")
throw new RuntimeException(s"Dataload failure for $dbName.$tableName, ${ex.getMessage}")
// In case of event related exception
case preEventEx: PreEventException =>
throw new AnalysisException(preEventEx.getMessage)
case ex: Exception =>
LOGGER.error(ex)
// update the load entry in table status file for changing the status to marked for delete
if (isUpdateTableStatusRequired) {
CarbonLoaderUtil.updateTableStatusForFailure(carbonLoadModel, uuid)
}
LOGGER.audit(s"Dataload failure for $dbName.$tableName. Please check the logs")
throw ex
} finally {
releaseConcurrentLoadLock(concurrentLoadLock, LOGGER)
// Once the data load is successful delete the unwanted partition files
try {
val partitionLocation = CarbonProperties.getStorePath + "/partition/" +
table.getDatabaseName + "/" +
table.getTableName + "/"
val fileType = FileFactory.getFileType(partitionLocation)
if (FileFactory.isFileExist(partitionLocation, fileType)) {
val file = FileFactory.getCarbonFile(partitionLocation, fileType)
CarbonUtil.deleteFoldersAndFiles(file)
}
} catch {
case ex: Exception =>
LOGGER.error(ex)
LOGGER.audit(s"Dataload failure for $dbName.$tableName. " +
"Problem deleting the partition folder")
throw ex
}
}
} catch {
case dle: DataLoadingException =>
LOGGER.audit(s"Dataload failed for $dbName.$tableName. " + dle.getMessage)
throw dle
case mce: MalformedCarbonCommandException =>
LOGGER.audit(s"Dataload failed for $dbName.$tableName. " + mce.getMessage)
throw mce
}
Seq.empty
}
private def acquireConcurrentLoadLock(): Option[ICarbonLock] = {
val isConcurrentLockRequired = table.getAllDimensions.asScala
.exists(cd => cd.hasEncoding(Encoding.DICTIONARY) &&
!cd.hasEncoding(Encoding.DIRECT_DICTIONARY))
if (isConcurrentLockRequired) {
var concurrentLoadLock: ICarbonLock = CarbonLockFactory.getCarbonLockObj(
table.getTableInfo().getOrCreateAbsoluteTableIdentifier(),
LockUsage.CONCURRENT_LOAD_LOCK)
val retryCount = CarbonLockUtil
.getLockProperty(CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK,
CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK_DEFAULT)
val maxTimeout = CarbonLockUtil
.getLockProperty(CarbonCommonConstants.MAX_TIMEOUT_FOR_CONCURRENT_LOCK,
CarbonCommonConstants.MAX_TIMEOUT_FOR_CONCURRENT_LOCK_DEFAULT)
if (!(isConcurrentLockRequired &&
concurrentLoadLock.lockWithRetries(retryCount, maxTimeout))) {
throw new RuntimeException(table.getDatabaseName + "." + table.getTableName +
" having dictionary column. so concurrent load is not supported")
}
return Some(concurrentLoadLock)
}
return None
}
private def releaseConcurrentLoadLock(concurrentLoadLock: Option[ICarbonLock],
LOGGER: LogService): Unit = {
if (concurrentLoadLock.isDefined) {
if (concurrentLoadLock.get.unlock()) {
LOGGER.info("concurrent_load lock for table" + table.getTablePath +
"has been released successfully")
} else {
LOGGER.error(
"Unable to unlock concurrent_load lock for table" + table.getTablePath);
}
}
}
private def loadDataUsingOnePass(
sparkSession: SparkSession,
carbonProperty: CarbonProperties,
carbonLoadModel: CarbonLoadModel,
columnar: Boolean,
partitionStatus: SegmentStatus,
hadoopConf: Configuration,
operationContext: OperationContext,
LOGGER: LogService): Seq[Row] = {
var rows = Seq.empty[Row]
val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
.getCarbonTableIdentifier
val dictFolderPath = CarbonTablePath.getMetadataPath(carbonLoadModel.getTablePath)
val dimensions = carbonTable.getDimensionByTableName(
carbonTable.getTableName).asScala.toArray
val colDictFilePath = carbonLoadModel.getColDictFilePath
if (!StringUtils.isEmpty(colDictFilePath)) {
carbonLoadModel.initPredefDictMap()
// generate predefined dictionary
GlobalDictionaryUtil.generatePredefinedColDictionary(
colDictFilePath,
carbonTableIdentifier,
dimensions,
carbonLoadModel,
sparkSession.sqlContext,
dictFolderPath)
}
if (!StringUtils.isEmpty(carbonLoadModel.getAllDictPath)) {
carbonLoadModel.initPredefDictMap()
GlobalDictionaryUtil
.generateDictionaryFromDictionaryFiles(sparkSession.sqlContext,
carbonLoadModel,
carbonTableIdentifier,
dictFolderPath,
dimensions,
carbonLoadModel.getAllDictPath)
}
// dictionaryServerClient dictionary generator
val dictionaryServerPort = carbonProperty
.getProperty(CarbonCommonConstants.DICTIONARY_SERVER_PORT,
CarbonCommonConstants.DICTIONARY_SERVER_PORT_DEFAULT)
val sparkDriverHost = sparkSession.sqlContext.sparkContext.
getConf.get("spark.driver.host")
carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
val carbonSecureModeDictServer = CarbonProperties.getInstance.
getProperty(CarbonCommonConstants.CARBON_SECURE_DICTIONARY_SERVER,
CarbonCommonConstants.CARBON_SECURE_DICTIONARY_SERVER_DEFAULT)
val sparkConf = sparkSession.sqlContext.sparkContext.getConf
// For testing.
// sparkConf.set("spark.authenticate", "true")
// sparkConf.set("spark.authenticate.secret", "secret")
val server: Option[DictionaryServer] = if (sparkConf.get("spark.authenticate", "false").
equalsIgnoreCase("true") && carbonSecureModeDictServer.toBoolean) {
val dictionaryServer = SecureDictionaryServer.getInstance(sparkConf,
sparkDriverHost.toString, dictionaryServerPort.toInt, carbonTable)
carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
carbonLoadModel.setDictionaryServerHost(dictionaryServer.getHost)
carbonLoadModel.setDictionaryServerSecretKey(dictionaryServer.getSecretKey)
carbonLoadModel.setDictionaryEncryptServerSecure(dictionaryServer.isEncryptSecureServer)
carbonLoadModel.setDictionaryServiceProvider(new SecureDictionaryServiceProvider())
sparkSession.sparkContext.addSparkListener(new SparkListener() {
override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
dictionaryServer.shutdown()
}
})
Some(dictionaryServer)
} else {
val dictionaryServer = NonSecureDictionaryServer
.getInstance(dictionaryServerPort.toInt, carbonTable)
carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
carbonLoadModel.setDictionaryServerHost(dictionaryServer.getHost)
carbonLoadModel.setDictionaryEncryptServerSecure(false)
carbonLoadModel
.setDictionaryServiceProvider(new NonSecureDictionaryServiceProvider(dictionaryServer
.getPort))
sparkSession.sparkContext.addSparkListener(new SparkListener() {
override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
dictionaryServer.shutdown()
}
})
Some(dictionaryServer)
}
val loadDataFrame = if (updateModel.isDefined) {
Some(getDataFrameWithTupleID())
} else {
dataFrame
}
if (carbonTable.isHivePartitionTable) {
try {
rows = loadDataWithPartition(
sparkSession,
carbonLoadModel,
hadoopConf,
loadDataFrame,
operationContext,
LOGGER)
} finally {
server match {
case Some(dictServer) =>
try {
dictServer.writeTableDictionary(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
.getCarbonTableIdentifier.getTableId)
} catch {
case _: Exception =>
throw new Exception("Dataload failed due to error while writing dictionary file!")
}
case _ =>
}
}
} else {
CarbonDataRDDFactory.loadCarbonData(
sparkSession.sqlContext,
carbonLoadModel,
columnar,
partitionStatus,
server,
isOverwriteTable,
hadoopConf,
loadDataFrame,
updateModel,
operationContext)
}
rows
}
private def loadData(
sparkSession: SparkSession,
carbonLoadModel: CarbonLoadModel,
columnar: Boolean,
partitionStatus: SegmentStatus,
hadoopConf: Configuration,
operationContext: OperationContext,
LOGGER: LogService): Seq[Row] = {
var rows = Seq.empty[Row]
val (dictionaryDataFrame, loadDataFrame) = if (updateModel.isDefined) {
val dataFrameWithTupleId: DataFrame = getDataFrameWithTupleID()
// getting all fields except tupleId field as it is not required in the value
val otherFields = CarbonScalaUtil.getAllFieldsWithoutTupleIdField(dataFrame.get.schema.fields)
// use dataFrameWithoutTupleId as dictionaryDataFrame
val dataFrameWithoutTupleId = dataFrame.get.select(otherFields: _*)
(Some(dataFrameWithoutTupleId), Some(dataFrameWithTupleId))
} else {
(dataFrame, dataFrame)
}
val table = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
if (!table.isChildDataMap) {
GlobalDictionaryUtil.generateGlobalDictionary(
sparkSession.sqlContext,
carbonLoadModel,
hadoopConf,
dictionaryDataFrame)
}
if (table.isHivePartitionTable) {
rows = loadDataWithPartition(
sparkSession,
carbonLoadModel,
hadoopConf,
loadDataFrame,
operationContext, LOGGER)
} else {
CarbonDataRDDFactory.loadCarbonData(
sparkSession.sqlContext,
carbonLoadModel,
columnar,
partitionStatus,
None,
isOverwriteTable,
hadoopConf,
loadDataFrame,
updateModel,
operationContext)
}
rows
}
/**
* Loads the data in a hive partition way. This method uses InsertIntoTable command to load data
* into partitioned data. The table relation would be converted to HadoopFSRelation to let spark
* handling the partitioning.
*/
private def loadDataWithPartition(
sparkSession: SparkSession,
carbonLoadModel: CarbonLoadModel,
hadoopConf: Configuration,
dataFrame: Option[DataFrame],
operationContext: OperationContext,
LOGGER: LogService): Seq[Row] = {
val table = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
val identifier = TableIdentifier(table.getTableName, Some(table.getDatabaseName))
val catalogTable: CatalogTable = logicalPartitionRelation.catalogTable.get
var timeStampformatString = carbonLoadModel.getTimestampformat
if (timeStampformatString.isEmpty) {
timeStampformatString = carbonLoadModel.getDefaultTimestampFormat
}
val timeStampFormat = new SimpleDateFormat(timeStampformatString)
var dateFormatString = carbonLoadModel.getDateFormat
if (dateFormatString.isEmpty) {
dateFormatString = carbonLoadModel.getDefaultDateFormat
}
val dateFormat = new SimpleDateFormat(dateFormatString)
// Clean up the alreday dropped partitioned data
SegmentFileStore.cleanSegments(table, null, false)
CarbonSession.threadSet("partition.operationcontext", operationContext)
// input data from csv files. Convert to logical plan
val allCols = new ArrayBuffer[String]()
allCols ++= table.getAllDimensions.asScala.map(_.getColName)
allCols ++= table.getAllMeasures.asScala.map(_.getColName)
var attributes =
StructType(
allCols.filterNot(_.equals(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)).map(
StructField(_, StringType))).toAttributes
var partitionsLen = 0
val sortScope = CarbonDataProcessorUtil.getSortScope(carbonLoadModel.getSortScope)
val partitionValues = if (partition.nonEmpty) {
partition.filter(_._2.nonEmpty).map { case (col, value) =>
catalogTable.schema.find(_.name.equalsIgnoreCase(col)) match {
case Some(c) =>
CarbonScalaUtil.convertToDateAndTimeFormats(
value.get,
c.dataType,
timeStampFormat,
dateFormat)
case None =>
throw new AnalysisException(s"$col is not a valid partition column in table ${
carbonLoadModel
.getDatabaseName
}.${ carbonLoadModel.getTableName }")
}
}.toArray
} else {
Array[String]()
}
var persistedRDD: Option[RDD[InternalRow]] = None
try {
val query: LogicalPlan = if (dataFrame.isDefined) {
val (rdd, dfAttributes) = if (updateModel.isDefined) {
// Get the updated query plan in case of update scenario
val updatedFrame = Dataset.ofRows(
sparkSession,
getLogicalQueryForUpdate(
sparkSession,
catalogTable,
dataFrame.get,
carbonLoadModel))
(updatedFrame.rdd, updatedFrame.schema)
} else {
if (partition.nonEmpty) {
val headers = carbonLoadModel.getCsvHeaderColumns.dropRight(partition.size)
val updatedHeader = headers ++ partition.keys.map(_.toLowerCase)
carbonLoadModel.setCsvHeader(updatedHeader.mkString(","))
carbonLoadModel.setCsvHeaderColumns(carbonLoadModel.getCsvHeader.split(","))
}
(dataFrame.get.rdd, dataFrame.get.schema)
}
val expectedColumns = {
val staticPartCols = partition.filter(_._2.isDefined).keySet
attributes.filterNot(a => staticPartCols.contains(a.name))
}
if (expectedColumns.length != dfAttributes.length) {
throw new AnalysisException(
s"Cannot insert into table $tableName because the number of columns are different: " +
s"need ${expectedColumns.length} columns, " +
s"but query has ${dfAttributes.length} columns.")
}
val nonPartitionBounds = expectedColumns.zipWithIndex.map(_._2).toArray
val partitionBounds = new Array[Int](partitionValues.length)
if (partition.nonEmpty) {
val nonPartitionSchemaLen = attributes.length - partition.size
var i = nonPartitionSchemaLen
var index = 0
var partIndex = 0
partition.values.foreach { p =>
if (p.isDefined) {
partitionBounds(partIndex) = nonPartitionSchemaLen + index
partIndex = partIndex + 1
} else {
nonPartitionBounds(i) = nonPartitionSchemaLen + index
i = i + 1
}
index = index + 1
}
}
val len = dfAttributes.length + partitionValues.length
val transRdd = rdd.map { f =>
val data = new Array[Any](len)
var i = 0
while (i < f.length) {
data(nonPartitionBounds(i)) = f.get(i)
i = i + 1
}
var j = 0
while (j < partitionBounds.length) {
data(partitionBounds(j)) = UTF8String.fromString(partitionValues(j))
j = j + 1
}
Row.fromSeq(data)
}
val (transformedPlan, partitions, persistedRDDLocal) =
transformQuery(
transRdd,
sparkSession,
carbonLoadModel,
partitionValues,
catalogTable,
attributes,
sortScope,
isDataFrame = true)
partitionsLen = partitions
persistedRDD = persistedRDDLocal
transformedPlan
} else {
val columnCount = carbonLoadModel.getCsvHeaderColumns.length
val rdd = CsvRDDHelper.csvFileScanRDD(
sparkSession,
model = carbonLoadModel,
hadoopConf).map(DataLoadProcessorStepOnSpark.toStringArrayRow(_, columnCount))
val (transformedPlan, partitions, persistedRDDLocal) =
transformQuery(
rdd.asInstanceOf[RDD[Row]],
sparkSession,
carbonLoadModel,
partitionValues,
catalogTable,
attributes,
sortScope,
isDataFrame = false)
partitionsLen = partitions
persistedRDD = persistedRDDLocal
transformedPlan
}
if (updateModel.isDefined) {
carbonLoadModel.setFactTimeStamp(updateModel.get.updatedTimeStamp)
}
// Create and ddd the segment to the tablestatus.
CarbonLoaderUtil.readAndUpdateLoadProgressInTableMeta(carbonLoadModel, isOverwriteTable)
val convertRelation = convertToLogicalRelation(
catalogTable,
sizeInBytes,
isOverwriteTable,
carbonLoadModel,
sparkSession,
operationContext)
val logicalPlan = if (sortScope == SortScopeOptions.SortScope.GLOBAL_SORT) {
var numPartitions =
CarbonDataProcessorUtil.getGlobalSortPartitions(carbonLoadModel.getGlobalSortPartitions)
if (numPartitions <= 0) {
numPartitions = partitionsLen
}
if (numPartitions > 0) {
Dataset.ofRows(sparkSession, query).repartition(numPartitions).logicalPlan
} else {
query
}
} else {
query
}
val convertedPlan =
CarbonReflectionUtils.getInsertIntoCommand(
table = convertRelation,
partition = partition,
query = logicalPlan,
overwrite = false,
ifPartitionNotExists = false)
sparkSession.sparkContext.setLocalProperty(EXECUTION_ID_KEY, null)
Dataset.ofRows(sparkSession, convertedPlan)
} catch {
case ex: Throwable =>
val (executorMessage, errorMessage) = CarbonScalaUtil.retrieveAndLogErrorMsg(ex, LOGGER)
if (updateModel.isDefined) {
CarbonScalaUtil.updateErrorInUpdateModel(updateModel.get, executorMessage)
}
LOGGER.info(errorMessage)
LOGGER.error(ex)
throw new Exception(errorMessage)
} finally {
CarbonSession.threadUnset("partition.operationcontext")
if (isOverwriteTable) {
DataMapStoreManager.getInstance().clearDataMaps(table.getAbsoluteTableIdentifier)
// Clean the overwriting segments if any.
SegmentFileStore.cleanSegments(
table,
null,
false)
}
if (partitionsLen > 1) {
// clean cache only if persisted and keeping unpersist non-blocking as non-blocking call
// will not have any functional impact as spark automatically monitors the cache usage on
// each node and drops out old data partitions in a least-recently used (LRU) fashion.
persistedRDD match {
case Some(rdd) => rdd.unpersist(false)
case _ =>
}
}
}
try {
carbonLoadModel.setFactTimeStamp(System.currentTimeMillis())
// Block compaction for table containing complex datatype
if (table.getTableInfo.getFactTable.getListOfColumns.asScala
.exists(m => m.getDataType.isComplexType)) {
LOGGER.warn("Compaction is skipped as table contains complex columns")
} else {
val compactedSegments = new util.ArrayList[String]()
// Trigger auto compaction
CarbonDataRDDFactory.handleSegmentMerging(
sparkSession.sqlContext,
carbonLoadModel,
table,
compactedSegments,
operationContext)
carbonLoadModel.setMergedSegmentIds(compactedSegments)
}
} catch {
case e: Exception =>
throw new Exception(
"Dataload is success. Auto-Compaction has failed. Please check logs.",
e)
}
val specs =
SegmentFileStore.getPartitionSpecs(carbonLoadModel.getSegmentId, carbonLoadModel.getTablePath)
if (specs != null) {
specs.asScala.map{ spec =>
Row(spec.getPartitions.asScala.mkString("/"), spec.getLocation.toString, spec.getUuid)
}
} else {
Seq.empty[Row]
}
}
/**
* Transform the rdd to logical plan as per the sortscope. If it is global sort scope then it
* will convert to sort logical plan otherwise project plan.
*/
private def transformQuery(rdd: RDD[Row],
sparkSession: SparkSession,
loadModel: CarbonLoadModel,
partitionValues: Array[String],
catalogTable: CatalogTable,
curAttributes: Seq[AttributeReference],
sortScope: SortScopeOptions.SortScope,
isDataFrame: Boolean): (LogicalPlan, Int, Option[RDD[InternalRow]]) = {
// Converts the data as per the loading steps before give it to writer or sorter
val updatedRdd = convertData(
rdd,
sparkSession,
loadModel,
isDataFrame,
partitionValues)
val catalogAttributes = catalogTable.schema.toAttributes
var attributes = curAttributes.map(a => {
catalogAttributes.find(_.name.equalsIgnoreCase(a.name)).get
})
attributes = attributes.map { attr =>
// Update attribute datatypes in case of dictionary columns, in case of dictionary columns
// datatype is always int
val column = table.getColumnByName(table.getTableName, attr.name)
if (column.hasEncoding(Encoding.DICTIONARY)) {
CarbonToSparkAdapater.createAttributeReference(attr.name,
IntegerType,
attr.nullable,
attr.metadata,
attr.exprId,
attr.qualifier,
attr)
} else if (attr.dataType == TimestampType || attr.dataType == DateType) {
CarbonToSparkAdapater.createAttributeReference(attr.name,
LongType,
attr.nullable,
attr.metadata,
attr.exprId,
attr.qualifier,
attr)
} else {
attr
}
}
// Only select the required columns
val output = if (partition.nonEmpty) {
val lowerCasePartition = partition.map { case (key, value) => (key.toLowerCase, value) }
catalogTable.schema.map { attr =>
attributes.find(_.name.equalsIgnoreCase(attr.name)).get
}.filter(attr => lowerCasePartition.getOrElse(attr.name.toLowerCase, None).isEmpty)
} else {
catalogTable.schema.map(f => attributes.find(_.name.equalsIgnoreCase(f.name)).get)
}
val partitionsLen = rdd.partitions.length
// If it is global sort scope then appl sort logical plan on the sort columns
if (sortScope == SortScopeOptions.SortScope.GLOBAL_SORT) {
// Because if the number of partitions greater than 1, there will be action operator(sample)
// in sortBy operator. So here we cache the rdd to avoid do input and convert again.
if (partitionsLen > 1) {
updatedRdd.persist(StorageLevel.fromString(
CarbonProperties.getInstance().getGlobalSortRddStorageLevel))
}
val child = Project(output, LogicalRDD(attributes, updatedRdd)(sparkSession))
val sortColumns = table.getSortColumns(table.getTableName)
val sortPlan =
Sort(
output.filter(f => sortColumns.contains(f.name)).map(SortOrder(_, Ascending)),
global = true,
child)
(sortPlan, partitionsLen, Some(updatedRdd))
} else {
(Project(output, LogicalRDD(attributes, updatedRdd)(sparkSession)), partitionsLen, None)
}
}
/**
* Convert the rdd as per steps of data loading inputprocessor step and converter step
* @param originRDD
* @param sparkSession
* @param model
* @param isDataFrame
* @param partitionValues
* @return
*/
private def convertData(
originRDD: RDD[Row],
sparkSession: SparkSession,
model: CarbonLoadModel,
isDataFrame: Boolean,
partitionValues: Array[String]): RDD[InternalRow] = {
val sc = sparkSession.sparkContext
val info =
model.getCarbonDataLoadSchema.getCarbonTable.getTableInfo.getFactTable.getPartitionInfo
info.setColumnSchemaList(new util.ArrayList[ColumnSchema](info.getColumnSchemaList))
val modelBroadcast = sc.broadcast(model)
val partialSuccessAccum = sc.accumulator(0, "Partial Success Accumulator")
val inputStepRowCounter = sc.accumulator(0, "Input Processor Accumulator")
// 1. Input
val convertRDD =
if (isDataFrame) {
originRDD.mapPartitions{rows =>
DataLoadProcessorStepOnSpark.toRDDIterator(rows, modelBroadcast)
}
} else {
// Append the partition columns in case of static partition scenario
val partitionLen = partitionValues.length
val len = model.getCsvHeaderColumns.length - partitionLen
originRDD.map{ row =>
val array = new Array[AnyRef](len + partitionLen)
var i = 0
while (i < len) {
array(i) = row.get(i).asInstanceOf[AnyRef]
i = i + 1
}
if (partitionLen > 0) {
System.arraycopy(partitionValues, 0, array, i, partitionLen)
}
array
}
}
val conf = sparkSession.sparkContext
.broadcast(new SerializableConfiguration(sparkSession.sessionState.newHadoopConf()))
val finalRDD = convertRDD.mapPartitionsWithIndex { case(index, rows) =>
DataTypeUtil.setDataTypeConverter(new SparkDataTypeConverterImpl)
ThreadLocalSessionInfo.setConfigurationToCurrentThread(conf.value.value)
DataLoadProcessorStepOnSpark.inputAndconvertFunc(
rows,
index,
modelBroadcast,
partialSuccessAccum,
inputStepRowCounter,
keepActualData = true)