forked from apache/carbondata
-
Notifications
You must be signed in to change notification settings - Fork 0
/
CarbonDataRDDFactory.scala
1091 lines (1030 loc) · 47.4 KB
/
CarbonDataRDDFactory.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.carbondata.spark.rdd
import java.io.File
import java.util
import java.util.concurrent._
import scala.collection.JavaConverters._
import scala.collection.mutable.ListBuffer
import scala.util.Random
import org.apache.commons.lang3.StringUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.mapreduce.Job
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
import org.apache.spark.{SparkEnv, TaskContext}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.rdd.{DataLoadCoalescedRDD, DataLoadPartitionCoalescer}
import org.apache.spark.sql.{CarbonEnv, DataFrame, Row, SQLContext}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.execution.command.{CompactionModel, ExecutionErrors, UpdateTableModel}
import org.apache.spark.sql.hive.DistributionUtil
import org.apache.spark.sql.optimizer.CarbonFilters
import org.apache.spark.sql.util.{CarbonException, SparkSQLUtil}
import org.apache.carbondata.common.constants.LoggerAction
import org.apache.carbondata.common.logging.LogServiceFactory
import org.apache.carbondata.core.constants.{CarbonCommonConstants, SortScopeOptions}
import org.apache.carbondata.core.datamap.{DataMapStoreManager, Segment}
import org.apache.carbondata.core.datamap.status.DataMapStatusManager
import org.apache.carbondata.core.datastore.block.{Distributable, TableBlockInfo}
import org.apache.carbondata.core.datastore.compression.CompressorFactory
import org.apache.carbondata.core.datastore.filesystem.CarbonFile
import org.apache.carbondata.core.datastore.impl.FileFactory
import org.apache.carbondata.core.exception.ConcurrentOperationException
import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnarFormatVersion, SegmentFileStore}
import org.apache.carbondata.core.metadata.schema.table.CarbonTable
import org.apache.carbondata.core.mutate.CarbonUpdateUtil
import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, ThreadLocalSessionInfo}
import org.apache.carbondata.core.util.path.CarbonTablePath
import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
import org.apache.carbondata.indexserver.{DistributedRDDUtils, IndexServer}
import org.apache.carbondata.processing.loading.FailureCauses
import org.apache.carbondata.processing.loading.csvinput.BlockDetails
import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostStatusUpdateEvent, LoadTablePreStatusUpdateEvent}
import org.apache.carbondata.processing.loading.exception.NoRetryException
import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
import org.apache.carbondata.processing.merger.{CarbonCompactionUtil, CarbonDataMergerUtil, CompactionType}
import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil}
import org.apache.carbondata.spark.{DataLoadResultImpl, _}
import org.apache.carbondata.spark.load._
import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, Util}
/**
* This is the factory class which can create different RDD depends on user needs.
*
*/
object CarbonDataRDDFactory {
private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
def handleCompactionForSystemLocking(sqlContext: SQLContext,
carbonLoadModel: CarbonLoadModel,
storeLocation: String,
compactionType: CompactionType,
carbonTable: CarbonTable,
compactedSegments: java.util.List[String],
compactionModel: CompactionModel,
operationContext: OperationContext): Unit = {
// taking system level lock at the mdt file location
var configuredMdtPath = CarbonProperties.getInstance().getProperty(
CarbonCommonConstants.CARBON_UPDATE_SYNC_FOLDER,
CarbonCommonConstants.CARBON_UPDATE_SYNC_FOLDER_DEFAULT).trim
configuredMdtPath = CarbonUtil.checkAndAppendFileSystemURIScheme(configuredMdtPath)
val lock = CarbonLockFactory.getSystemLevelCarbonLockObj(
configuredMdtPath + CarbonCommonConstants.FILE_SEPARATOR +
CarbonCommonConstants.SYSTEM_LEVEL_COMPACTION_LOCK_FOLDER,
LockUsage.SYSTEMLEVEL_COMPACTION_LOCK)
if (lock.lockWithRetries()) {
LOGGER.info(s"Acquired the compaction lock for table ${ carbonLoadModel.getDatabaseName }" +
s".${ carbonLoadModel.getTableName }")
try {
startCompactionThreads(
sqlContext,
carbonLoadModel,
storeLocation,
compactionModel,
lock,
compactedSegments,
operationContext
)
} catch {
case e: Exception =>
LOGGER.error(s"Exception in start compaction thread. ${ e.getMessage }")
lock.unlock()
// if the compaction is a blocking call then only need to throw the exception.
if (compactionModel.isDDLTrigger) {
throw e
}
}
} else {
LOGGER.error("Not able to acquire the compaction lock for table " +
s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
CarbonCompactionUtil
.createCompactionRequiredFile(carbonTable.getMetadataPath, compactionType)
// throw exception only in case of DDL trigger.
if (compactionModel.isDDLTrigger) {
CarbonException.analysisException(
s"Compaction is in progress, compaction request for table " +
s"${carbonLoadModel.getDatabaseName}.${carbonLoadModel.getTableName}" +
" is in queue.")
} else {
LOGGER.error("Compaction is in progress, compaction request for table " +
s"${carbonLoadModel.getDatabaseName}.${carbonLoadModel.getTableName}" +
" is in queue.")
}
}
}
def startCompactionThreads(sqlContext: SQLContext,
carbonLoadModel: CarbonLoadModel,
storeLocation: String,
compactionModel: CompactionModel,
compactionLock: ICarbonLock,
compactedSegments: java.util.List[String],
operationContext: OperationContext): Unit = {
val executor: ExecutorService = Executors.newFixedThreadPool(1)
// update the updated table status.
if (compactionModel.compactionType != CompactionType.IUD_UPDDEL_DELTA) {
// update the updated table status. For the case of Update Delta Compaction the Metadata
// is filled in LoadModel, no need to refresh.
carbonLoadModel.readAndSetLoadMetadataDetails()
}
val compactionThread = new Thread {
override def run(): Unit = {
val compactor = CompactionFactory.getCompactor(
carbonLoadModel,
compactionModel,
executor,
sqlContext,
storeLocation,
compactedSegments,
operationContext)
try {
// compaction status of the table which is triggered by the user.
var triggeredCompactionStatus = false
var exception: Exception = null
try {
compactor.executeCompaction()
triggeredCompactionStatus = true
} catch {
case e: Exception =>
LOGGER.error(s"Exception in compaction thread ${ e.getMessage }")
exception = e
}
// continue in case of exception also, check for all the tables.
val isConcurrentCompactionAllowed = CarbonProperties.getInstance().getProperty(
CarbonCommonConstants.ENABLE_CONCURRENT_COMPACTION,
CarbonCommonConstants.DEFAULT_ENABLE_CONCURRENT_COMPACTION
).equalsIgnoreCase("true")
if (!isConcurrentCompactionAllowed) {
LOGGER.info("System level compaction lock is enabled.")
val skipCompactionTables = ListBuffer[CarbonTableIdentifier]()
var tableForCompaction = CarbonCompactionUtil.getNextTableToCompact(
CarbonEnv.getInstance(sqlContext.sparkSession).carbonMetaStore
.listAllTables(sqlContext.sparkSession).toArray,
skipCompactionTables.toList.asJava)
while (null != tableForCompaction) {
LOGGER.info("Compaction request has been identified for table " +
s"${ tableForCompaction.getDatabaseName }." +
s"${ tableForCompaction.getTableName}")
val table: CarbonTable = tableForCompaction
val metadataPath = table.getMetadataPath
val compactionType = CarbonCompactionUtil.determineCompactionType(metadataPath)
val newCarbonLoadModel = prepareCarbonLoadModel(table)
val compactionSize = CarbonDataMergerUtil
.getCompactionSize(CompactionType.MAJOR, carbonLoadModel)
val newcompactionModel = CompactionModel(
compactionSize,
compactionType,
table,
compactionModel.isDDLTrigger,
CarbonFilters.getCurrentPartitions(sqlContext.sparkSession,
TableIdentifier(table.getTableName,
Some(table.getDatabaseName))), None)
// proceed for compaction
try {
CompactionFactory.getCompactor(
newCarbonLoadModel,
newcompactionModel,
executor,
sqlContext,
storeLocation,
compactedSegments,
operationContext).executeCompaction()
} catch {
case e: Exception =>
LOGGER.error("Exception in compaction thread for table " +
s"${ tableForCompaction.getDatabaseName }." +
s"${ tableForCompaction.getTableName }")
// not handling the exception. only logging as this is not the table triggered
// by user.
} finally {
// delete the compaction required file in case of failure or success also.
if (!CarbonCompactionUtil
.deleteCompactionRequiredFile(metadataPath, compactionType)) {
// if the compaction request file is not been able to delete then
// add those tables details to the skip list so that it wont be considered next.
skipCompactionTables.+=:(tableForCompaction.getCarbonTableIdentifier)
LOGGER.error("Compaction request file can not be deleted for table " +
s"${ tableForCompaction.getDatabaseName }." +
s"${ tableForCompaction.getTableName }")
}
}
// ********* check again for all the tables.
tableForCompaction = CarbonCompactionUtil.getNextTableToCompact(
CarbonEnv.getInstance(sqlContext.sparkSession).carbonMetaStore
.listAllTables(sqlContext.sparkSession).toArray,
skipCompactionTables.asJava)
}
}
// Remove compacted segments from executor cache.
if (CarbonProperties.getInstance().isDistributedPruningEnabled(
carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName)) {
try {
IndexServer.getClient.invalidateSegmentCache(carbonLoadModel
.getCarbonDataLoadSchema.getCarbonTable,
compactedSegments.asScala.toArray,
SparkSQLUtil.getTaskGroupId(sqlContext.sparkSession))
} catch {
case ex: Exception =>
LOGGER.warn(s"Clear cache job has failed for ${carbonLoadModel
.getDatabaseName}.${carbonLoadModel.getTableName}", ex)
}
}
// giving the user his error for telling in the beeline if his triggered table
// compaction is failed.
if (!triggeredCompactionStatus) {
throw new Exception("Exception in compaction " + exception.getMessage)
}
} finally {
executor.shutdownNow()
compactor.deletePartialLoadsInCompaction()
if (compactionModel.compactionType != CompactionType.IUD_UPDDEL_DELTA) {
compactionLock.unlock()
}
}
}
}
// calling the run method of a thread to make the call as blocking call.
// in the future we may make this as concurrent.
compactionThread.run()
}
private def prepareCarbonLoadModel(
table: CarbonTable
): CarbonLoadModel = {
val loadModel = new CarbonLoadModel
loadModel.setTableName(table.getTableName)
val dataLoadSchema = new CarbonDataLoadSchema(table)
// Need to fill dimension relation
loadModel.setCarbonDataLoadSchema(dataLoadSchema)
loadModel.setTableName(table.getCarbonTableIdentifier.getTableName)
loadModel.setDatabaseName(table.getCarbonTableIdentifier.getDatabaseName)
loadModel.setTablePath(table.getTablePath)
loadModel.setCarbonTransactionalTable(table.isTransactionalTable)
loadModel.readAndSetLoadMetadataDetails()
val loadStartTime = CarbonUpdateUtil.readCurrentTime()
loadModel.setFactTimeStamp(loadStartTime)
val columnCompressor = table.getTableInfo.getFactTable.getTableProperties.asScala
.getOrElse(CarbonCommonConstants.COMPRESSOR,
CompressorFactory.getInstance().getCompressor.getName)
loadModel.setColumnCompressor(columnCompressor)
loadModel
}
def loadCarbonData(
sqlContext: SQLContext,
carbonLoadModel: CarbonLoadModel,
columnar: Boolean,
partitionStatus: SegmentStatus = SegmentStatus.SUCCESS,
overwriteTable: Boolean,
hadoopConf: Configuration,
dataFrame: Option[DataFrame] = None,
updateModel: Option[UpdateTableModel] = None,
operationContext: OperationContext): LoadMetadataDetails = {
// Check if any load need to be deleted before loading new data
val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
var status: Array[(String, (LoadMetadataDetails, ExecutionErrors))] = null
var res: Array[List[(String, (LoadMetadataDetails, ExecutionErrors))]] = null
// create new segment folder in carbon store
if (updateModel.isEmpty && carbonLoadModel.isCarbonTransactionalTable) {
CarbonLoaderUtil.checkAndCreateCarbonDataLocation(carbonLoadModel.getSegmentId, carbonTable)
}
var loadStatus = SegmentStatus.SUCCESS
var errorMessage: String = "DataLoad failure"
var executorMessage: String = ""
val isSortTable = carbonTable.getNumberOfSortColumns > 0
val sortScope = CarbonDataProcessorUtil.getSortScope(carbonLoadModel.getSortScope)
val segmentLock = CarbonLockFactory.getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier,
CarbonTablePath.addSegmentPrefix(carbonLoadModel.getSegmentId) + LockUsage.LOCK)
try {
if (!carbonLoadModel.isCarbonTransactionalTable || segmentLock.lockWithRetries()) {
if (updateModel.isDefined) {
res = loadDataFrameForUpdate(
sqlContext,
dataFrame,
carbonLoadModel,
updateModel,
carbonTable,
hadoopConf)
res.foreach { resultOfSeg =>
resultOfSeg.foreach { resultOfBlock =>
if (resultOfBlock._2._1.getSegmentStatus == SegmentStatus.LOAD_FAILURE) {
loadStatus = SegmentStatus.LOAD_FAILURE
if (resultOfBlock._2._2.failureCauses == FailureCauses.NONE) {
updateModel.get.executorErrors.failureCauses = FailureCauses.EXECUTOR_FAILURE
updateModel.get.executorErrors.errorMsg = "Failure in the Executor."
} else {
updateModel.get.executorErrors = resultOfBlock._2._2
}
} else if (resultOfBlock._2._1.getSegmentStatus ==
SegmentStatus.LOAD_PARTIAL_SUCCESS) {
loadStatus = SegmentStatus.LOAD_PARTIAL_SUCCESS
updateModel.get.executorErrors.failureCauses = resultOfBlock._2._2.failureCauses
updateModel.get.executorErrors.errorMsg = resultOfBlock._2._2.errorMsg
}
}
}
} else {
status = if (dataFrame.isEmpty && isSortTable &&
carbonLoadModel.getRangePartitionColumn != null &&
(sortScope.equals(SortScopeOptions.SortScope.GLOBAL_SORT) ||
sortScope.equals(SortScopeOptions.SortScope.LOCAL_SORT))) {
DataLoadProcessBuilderOnSpark
.loadDataUsingRangeSort(sqlContext.sparkSession, carbonLoadModel, hadoopConf)
} else if (isSortTable && sortScope.equals(SortScopeOptions.SortScope.GLOBAL_SORT)) {
DataLoadProcessBuilderOnSpark.loadDataUsingGlobalSort(sqlContext.sparkSession,
dataFrame, carbonLoadModel, hadoopConf)
} else if (dataFrame.isDefined) {
loadDataFrame(sqlContext, dataFrame, carbonLoadModel)
} else {
loadDataFile(sqlContext, carbonLoadModel, hadoopConf)
}
val newStatusMap = scala.collection.mutable.Map.empty[String, SegmentStatus]
if (status.nonEmpty) {
status.foreach { eachLoadStatus =>
val state = newStatusMap.get(eachLoadStatus._1)
state match {
case Some(SegmentStatus.LOAD_FAILURE) =>
newStatusMap.put(eachLoadStatus._1, eachLoadStatus._2._1.getSegmentStatus)
case Some(SegmentStatus.LOAD_PARTIAL_SUCCESS)
if eachLoadStatus._2._1.getSegmentStatus ==
SegmentStatus.SUCCESS =>
newStatusMap.put(eachLoadStatus._1, eachLoadStatus._2._1.getSegmentStatus)
case _ =>
newStatusMap.put(eachLoadStatus._1, eachLoadStatus._2._1.getSegmentStatus)
}
}
newStatusMap.foreach {
case (key, value) =>
if (value == SegmentStatus.LOAD_FAILURE) {
loadStatus = SegmentStatus.LOAD_FAILURE
} else if (value == SegmentStatus.LOAD_PARTIAL_SUCCESS &&
loadStatus != SegmentStatus.LOAD_FAILURE) {
loadStatus = SegmentStatus.LOAD_PARTIAL_SUCCESS
}
}
} else {
// if no value is there in data load, make load status Success
// and data load flow executes
if (dataFrame.isDefined && updateModel.isEmpty) {
val rdd = dataFrame.get.rdd
if (rdd.partitions == null || rdd.partitions.length == 0) {
LOGGER.warn("DataLoading finished. No data was loaded.")
loadStatus = SegmentStatus.SUCCESS
}
} else {
loadStatus = SegmentStatus.LOAD_FAILURE
}
}
if (loadStatus != SegmentStatus.LOAD_FAILURE &&
partitionStatus == SegmentStatus.LOAD_PARTIAL_SUCCESS) {
loadStatus = partitionStatus
}
}
}
} catch {
case ex: Throwable =>
loadStatus = SegmentStatus.LOAD_FAILURE
val (extrMsgLocal, errorMsgLocal) = CarbonScalaUtil.retrieveAndLogErrorMsg(ex, LOGGER)
executorMessage = extrMsgLocal
errorMessage = errorMsgLocal
LOGGER.info(errorMessage)
LOGGER.error(ex)
} finally {
segmentLock.unlock()
}
// handle the status file updation for the update cmd.
if (updateModel.isDefined) {
if (loadStatus == SegmentStatus.LOAD_FAILURE) {
CarbonScalaUtil.updateErrorInUpdateModel(updateModel.get, executorMessage)
return null
} else if (loadStatus == SegmentStatus.LOAD_PARTIAL_SUCCESS &&
updateModel.get.executorErrors.failureCauses == FailureCauses.BAD_RECORDS &&
carbonLoadModel.getBadRecordsAction.split(",")(1) == LoggerAction.FAIL.name) {
return null
} else {
// in success case handle updation of the table status file.
// success case.
val segmentDetails = new util.HashSet[Segment]()
var resultSize = 0
res.foreach { resultOfSeg =>
resultSize = resultSize + resultOfSeg.size
resultOfSeg.foreach { resultOfBlock =>
segmentDetails.add(new Segment(resultOfBlock._2._1.getLoadName))
}
}
val segmentFiles = updateSegmentFiles(carbonTable, segmentDetails, updateModel.get)
// this means that the update doesnt have any records to update so no need to do table
// status file updation.
if (resultSize == 0) {
return null
}
if (!CarbonUpdateUtil.updateTableMetadataStatus(
segmentDetails,
carbonTable,
updateModel.get.updatedTimeStamp + "",
true,
new util.ArrayList[Segment](0),
new util.ArrayList[Segment](segmentFiles), "")) {
LOGGER.error("Data update failed due to failure in table status updation.")
updateModel.get.executorErrors.errorMsg = errorMessage
updateModel.get.executorErrors.failureCauses = FailureCauses
.STATUS_FILE_UPDATION_FAILURE
return null
}
// code to handle Pre-Priming cache for update command
if (!segmentFiles.isEmpty) {
val segmentsToPrePrime = segmentFiles.asScala.map(iterator => iterator.getSegmentNo).toSeq
DistributedRDDUtils
.triggerPrepriming(sqlContext.sparkSession, carbonTable, segmentsToPrePrime,
operationContext, hadoopConf, segmentsToPrePrime.toList)
}
}
return null
}
val uniqueTableStatusId = Option(operationContext.getProperty("uuid")).getOrElse("")
.asInstanceOf[String]
if (loadStatus == SegmentStatus.LOAD_FAILURE) {
// update the load entry in table status file for changing the status to marked for delete
CarbonLoaderUtil.updateTableStatusForFailure(carbonLoadModel, uniqueTableStatusId)
LOGGER.info("********starting clean up**********")
if (carbonLoadModel.isCarbonTransactionalTable) {
// delete segment is applicable for transactional table
CarbonLoaderUtil.deleteSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)
clearDataMapFiles(carbonTable, carbonLoadModel.getSegmentId)
}
LOGGER.info("********clean up done**********")
LOGGER.warn("Cannot write load metadata file as data load failed")
throw new Exception(errorMessage)
} else {
// check if data load fails due to bad record and throw data load failure due to
// bad record exception
if (loadStatus == SegmentStatus.LOAD_PARTIAL_SUCCESS &&
status(0)._2._2.failureCauses == FailureCauses.BAD_RECORDS &&
carbonLoadModel.getBadRecordsAction.split(",")(1) == LoggerAction.FAIL.name) {
// update the load entry in table status file for changing the status to marked for delete
CarbonLoaderUtil.updateTableStatusForFailure(carbonLoadModel, uniqueTableStatusId)
LOGGER.info("********starting clean up**********")
if (carbonLoadModel.isCarbonTransactionalTable) {
// delete segment is applicable for transactional table
CarbonLoaderUtil.deleteSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)
clearDataMapFiles(carbonTable, carbonLoadModel.getSegmentId)
}
LOGGER.info("********clean up done**********")
throw new Exception(status(0)._2._2.errorMsg)
}
// as no record loaded in new segment, new segment should be deleted
val newEntryLoadStatus =
if (carbonLoadModel.isCarbonTransactionalTable &&
!carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isChildTableForMV &&
!CarbonLoaderUtil.isValidSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)) {
LOGGER.warn("Cannot write load metadata file as there is no data to load")
SegmentStatus.MARKED_FOR_DELETE
} else {
loadStatus
}
val segmentFileName =
SegmentFileStore.writeSegmentFile(carbonTable, carbonLoadModel.getSegmentId,
String.valueOf(carbonLoadModel.getFactTimeStamp))
SegmentFileStore.updateTableStatusFile(
carbonTable,
carbonLoadModel.getSegmentId,
segmentFileName,
carbonTable.getCarbonTableIdentifier.getTableId,
new SegmentFileStore(carbonTable.getTablePath, segmentFileName))
operationContext.setProperty(carbonTable.getTableUniqueName + "_Segment",
carbonLoadModel.getSegmentId)
val loadTablePreStatusUpdateEvent: LoadTablePreStatusUpdateEvent =
new LoadTablePreStatusUpdateEvent(
carbonTable.getCarbonTableIdentifier,
carbonLoadModel)
OperationListenerBus.getInstance().fireEvent(loadTablePreStatusUpdateEvent, operationContext)
val (done, writtenSegment) =
updateTableStatus(
status,
carbonLoadModel,
newEntryLoadStatus,
overwriteTable,
segmentFileName,
uniqueTableStatusId)
val loadTablePostStatusUpdateEvent: LoadTablePostStatusUpdateEvent =
new LoadTablePostStatusUpdateEvent(carbonLoadModel)
val commitComplete = try {
OperationListenerBus.getInstance()
.fireEvent(loadTablePostStatusUpdateEvent, operationContext)
true
} catch {
case ex: Exception =>
LOGGER.error("Problem while committing data maps", ex)
false
}
if (!done || !commitComplete) {
CarbonLoaderUtil.updateTableStatusForFailure(carbonLoadModel, uniqueTableStatusId)
LOGGER.info("********starting clean up**********")
if (carbonLoadModel.isCarbonTransactionalTable) {
// delete segment is applicable for transactional table
CarbonLoaderUtil.deleteSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)
// delete corresponding segment file from metadata
val segmentFile = CarbonTablePath.getSegmentFilesLocation(carbonLoadModel.getTablePath) +
File.separator + segmentFileName
FileFactory.deleteFile(segmentFile)
clearDataMapFiles(carbonTable, carbonLoadModel.getSegmentId)
}
LOGGER.info("********clean up done**********")
LOGGER.error("Data load failed due to failure in table status updation.")
throw new Exception("Data load failed due to failure in table status updation.")
}
if (SegmentStatus.LOAD_PARTIAL_SUCCESS == loadStatus) {
LOGGER.info("Data load is partially successful for " +
s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
} else {
LOGGER.info("Data load is successful for " +
s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
}
// code to handle Pre-Priming cache for loading
if (!StringUtils.isEmpty(carbonLoadModel.getSegmentId)) {
DistributedRDDUtils.triggerPrepriming(sqlContext.sparkSession, carbonTable, Seq(),
operationContext, hadoopConf, List(carbonLoadModel.getSegmentId))
}
try {
// compaction handling
if (carbonTable.isHivePartitionTable) {
carbonLoadModel.setFactTimeStamp(System.currentTimeMillis())
}
val compactedSegments = new util.ArrayList[String]()
handleSegmentMerging(sqlContext,
carbonLoadModel
.getCopyWithPartition(carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter),
carbonTable,
compactedSegments,
operationContext)
carbonLoadModel.setMergedSegmentIds(compactedSegments)
writtenSegment
} catch {
case e: Exception =>
LOGGER.error(
"Auto-Compaction has failed. Ignoring this exception because the" +
" load is passed.", e)
writtenSegment
}
}
}
/**
* clear datamap files for segment
*/
def clearDataMapFiles(carbonTable: CarbonTable, segmentId: String): Unit = {
try {
val segments = List(new Segment(segmentId)).asJava
DataMapStoreManager.getInstance().getAllDataMap(carbonTable).asScala
.filter(_.getDataMapSchema.isIndexDataMap)
.foreach(_.deleteDatamapData(segments))
} catch {
case ex : Exception =>
LOGGER.error(s"Failed to clear datamap files for" +
s" ${carbonTable.getDatabaseName}.${carbonTable.getTableName}")
}
}
/**
* Add and update the segment files. In case of update scenario the carbonindex files are written
* to the same segment so we need to update old segment file. So this ethod writes the latest data
* to new segment file and merges this file old file to get latest updated files.
* @param carbonTable
* @param segmentDetails
* @return
*/
private def updateSegmentFiles(
carbonTable: CarbonTable,
segmentDetails: util.HashSet[Segment],
updateModel: UpdateTableModel) = {
val metadataDetails =
SegmentStatusManager.readTableStatusFile(
CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath))
val segmentFiles = segmentDetails.asScala.map { seg =>
val load =
metadataDetails.find(_.getLoadName.equals(seg.getSegmentNo)).get
val segmentFile = load.getSegmentFile
var segmentFiles: Seq[CarbonFile] = Seq.empty[CarbonFile]
val file = SegmentFileStore.writeSegmentFile(
carbonTable,
seg.getSegmentNo,
String.valueOf(System.currentTimeMillis()),
load.getPath)
if (segmentFile != null) {
segmentFiles ++= FileFactory.getCarbonFile(
SegmentFileStore.getSegmentFilePath(carbonTable.getTablePath, segmentFile)) :: Nil
}
val updatedSegFile = if (file != null) {
val carbonFile = FileFactory.getCarbonFile(
SegmentFileStore.getSegmentFilePath(carbonTable.getTablePath, file))
segmentFiles ++= carbonFile :: Nil
val mergedSegFileName = SegmentFileStore.genSegmentFileName(
seg.getSegmentNo,
updateModel.updatedTimeStamp.toString)
SegmentFileStore.mergeSegmentFiles(
mergedSegFileName,
CarbonTablePath.getSegmentFilesLocation(carbonTable.getTablePath),
segmentFiles.toArray)
carbonFile.delete()
mergedSegFileName + CarbonTablePath.SEGMENT_EXT
} else {
null
}
new Segment(seg.getSegmentNo, updatedSegFile)
}.filter(_.getSegmentFileName != null).asJava
segmentFiles
}
/**
* If data load is triggered by UPDATE query, this func will execute the update
* TODO: move it to a separate update command
*/
private def loadDataFrameForUpdate(
sqlContext: SQLContext,
dataFrame: Option[DataFrame],
carbonLoadModel: CarbonLoadModel,
updateModel: Option[UpdateTableModel],
carbonTable: CarbonTable,
hadoopConf: Configuration): Array[List[(String, (LoadMetadataDetails, ExecutionErrors))]] = {
val segmentUpdateParallelism = CarbonProperties.getInstance().getParallelismForSegmentUpdate
val updateRdd = dataFrame.get.rdd
// return directly if no rows to update
val noRowsToUpdate = updateRdd.isEmpty()
if (noRowsToUpdate) {
Array[List[(String, (LoadMetadataDetails, ExecutionErrors))]]()
} else {
// splitting as (key, value) i.e., (segment, updatedRows)
val keyRDD = updateRdd.map(row =>
(row.get(row.size - 1).toString, Row(row.toSeq.slice(0, row.size - 1): _*)))
val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(
carbonTable.getMetadataPath)
.filter(lmd => lmd.getSegmentStatus.equals(SegmentStatus.LOAD_PARTIAL_SUCCESS) ||
lmd.getSegmentStatus.equals(SegmentStatus.SUCCESS))
val segments = loadMetadataDetails.map(f => new Segment(f.getLoadName, f.getSegmentFile))
val segmentIdIndex = segments.map(_.getSegmentNo).zipWithIndex.toMap
val segmentId2maxTaskNo = segments.map { seg =>
(seg.getSegmentNo,
CarbonUpdateUtil.getLatestTaskIdForSegment(seg, carbonLoadModel.getTablePath))
}.toMap
class SegmentPartitioner(segIdIndex: Map[String, Int], parallelism: Int)
extends org.apache.spark.Partitioner {
override def numPartitions: Int = segmentIdIndex.size * parallelism
override def getPartition(key: Any): Int = {
val segId = key.asInstanceOf[String]
segmentIdIndex(segId) * parallelism + Random.nextInt(parallelism)
}
}
val partitionByRdd = keyRDD.partitionBy(
new SegmentPartitioner(segmentIdIndex, segmentUpdateParallelism))
// because partitionId=segmentIdIndex*parallelism+RandomPart and RandomPart<parallelism,
// so segmentIdIndex=partitionId/parallelism, this has been verified.
val conf = SparkSQLUtil.broadCastHadoopConf(sqlContext.sparkSession.sparkContext, hadoopConf)
partitionByRdd.map(_._2).mapPartitions { partition =>
ThreadLocalSessionInfo.setConfigurationToCurrentThread(conf.value.value)
val partitionId = TaskContext.getPartitionId()
val segIdIndex = partitionId / segmentUpdateParallelism
val randomPart = partitionId - segIdIndex * segmentUpdateParallelism
val segId = segments(segIdIndex)
val newTaskNo = segmentId2maxTaskNo(segId.getSegmentNo) + randomPart + 1
List(triggerDataLoadForSegment(
carbonLoadModel,
updateModel,
segId.getSegmentNo,
newTaskNo,
partition).toList).toIterator
}.collect()
}
}
/**
* TODO: move it to a separate update command
*/
private def triggerDataLoadForSegment(
carbonLoadModel: CarbonLoadModel,
updateModel: Option[UpdateTableModel],
key: String,
taskNo: Long,
iter: Iterator[Row]): Iterator[(String, (LoadMetadataDetails, ExecutionErrors))] = {
val rddResult = new updateResultImpl()
val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
val resultIter = new Iterator[(String, (LoadMetadataDetails, ExecutionErrors))] {
val loadMetadataDetails = new LoadMetadataDetails
val executionErrors = ExecutionErrors(FailureCauses.NONE, "")
var uniqueLoadStatusId = ""
try {
val segId = key
val index = taskNo
uniqueLoadStatusId = carbonLoadModel.getTableName +
CarbonCommonConstants.UNDERSCORE +
(index + "_0")
loadMetadataDetails.setLoadName(segId)
loadMetadataDetails.setSegmentStatus(SegmentStatus.LOAD_FAILURE)
carbonLoadModel.setSegmentId(segId)
carbonLoadModel.setTaskNo(String.valueOf(index))
carbonLoadModel.setFactTimeStamp(updateModel.get.updatedTimeStamp)
loadMetadataDetails.setSegmentStatus(SegmentStatus.SUCCESS)
UpdateDataLoad.DataLoadForUpdate(segId,
index,
iter,
carbonLoadModel,
loadMetadataDetails)
} catch {
case e: NoRetryException =>
loadMetadataDetails
.setSegmentStatus(SegmentStatus.LOAD_PARTIAL_SUCCESS)
executionErrors.failureCauses = FailureCauses.BAD_RECORDS
executionErrors.errorMsg = e.getMessage
LOGGER.info("Bad Record Found")
case e: Exception =>
LOGGER.info("DataLoad failure")
LOGGER.error(e)
throw e
}
var finished = false
override def hasNext: Boolean = !finished
override def next(): (String, (LoadMetadataDetails, ExecutionErrors)) = {
finished = true
rddResult
.getKey(uniqueLoadStatusId,
(loadMetadataDetails, executionErrors))
}
}
resultIter
}
/**
* Trigger compaction after data load
*/
def handleSegmentMerging(
sqlContext: SQLContext,
carbonLoadModel: CarbonLoadModel,
carbonTable: CarbonTable,
compactedSegments: java.util.List[String],
operationContext: OperationContext): Unit = {
LOGGER.info(s"compaction need status is" +
s" ${ CarbonDataMergerUtil.checkIfAutoLoadMergingRequired(carbonTable) }")
if (CarbonDataMergerUtil.checkIfAutoLoadMergingRequired(carbonTable)) {
val compactionSize = 0
val isCompactionTriggerByDDl = false
val compactionModel = CompactionModel(
compactionSize,
CompactionType.MINOR,
carbonTable,
isCompactionTriggerByDDl,
CarbonFilters.getCurrentPartitions(sqlContext.sparkSession,
TableIdentifier(carbonTable.getTableName,
Some(carbonTable.getDatabaseName))), None)
var storeLocation = ""
val configuredStore = Util.getConfiguredLocalDirs(SparkEnv.get.conf)
if (null != configuredStore && configuredStore.nonEmpty) {
storeLocation = configuredStore(Random.nextInt(configuredStore.length))
}
if (storeLocation == null) {
storeLocation = System.getProperty("java.io.tmpdir")
}
storeLocation = storeLocation + "/carbonstore/" + System.nanoTime()
val isConcurrentCompactionAllowed = CarbonProperties.getInstance().getProperty(
CarbonCommonConstants.ENABLE_CONCURRENT_COMPACTION,
CarbonCommonConstants.DEFAULT_ENABLE_CONCURRENT_COMPACTION
).equalsIgnoreCase("true")
if (!isConcurrentCompactionAllowed) {
handleCompactionForSystemLocking(sqlContext,
carbonLoadModel,
storeLocation,
CompactionType.MINOR,
carbonTable,
compactedSegments,
compactionModel,
operationContext
)
} else {
val lock = CarbonLockFactory.getCarbonLockObj(
carbonTable.getAbsoluteTableIdentifier,
LockUsage.COMPACTION_LOCK)
val updateLock = CarbonLockFactory.getCarbonLockObj(carbonTable
.getAbsoluteTableIdentifier, LockUsage.UPDATE_LOCK)
try {
if (updateLock.lockWithRetries(3, 3)) {
if (lock.lockWithRetries()) {
LOGGER.info("Acquired the compaction lock.")
startCompactionThreads(sqlContext,
carbonLoadModel,
storeLocation,
compactionModel,
lock,
compactedSegments,
operationContext
)
} else {
LOGGER.error("Not able to acquire the compaction lock for table " +
s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName}")
}
} else {
throw new ConcurrentOperationException(carbonTable, "update", "compaction")
}
} catch {
case e: Exception =>
LOGGER.error(s"Exception in start compaction thread.", e)
lock.unlock()
throw e
} finally {
updateLock.unlock()
}
}
}
}
/**
* Update table status file after data loading
* @param status status collected from each task
* @param carbonLoadModel load model used for loading
* @param newEntryLoadStatus segment status to set in the metadata
* @param overwriteTable true the operation is overwrite
* @param segmentFileName segment file name
* @param uuid uuid for the table status file name
* @return whether operation success and
* the segment metadata that written into the segment status file
*/
private def updateTableStatus(
status: Array[(String, (LoadMetadataDetails, ExecutionErrors))],
carbonLoadModel: CarbonLoadModel,
newEntryLoadStatus: SegmentStatus,
overwriteTable: Boolean,
segmentFileName: String,
uuid: String = ""): (Boolean, LoadMetadataDetails) = {
val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
val metadataDetails = if (status != null && status.size > 0 && status(0) != null) {
status(0)._2._1
} else {
new LoadMetadataDetails
}
metadataDetails.setSegmentFile(segmentFileName)
CarbonLoaderUtil.populateNewLoadMetaEntry(
metadataDetails,
newEntryLoadStatus,
carbonLoadModel.getFactTimeStamp,
true)
CarbonLoaderUtil
.addDataIndexSizeIntoMetaEntry(metadataDetails, carbonLoadModel.getSegmentId, carbonTable)
if (!carbonLoadModel.isCarbonTransactionalTable && overwriteTable) {
CarbonLoaderUtil.deleteNonTransactionalTableForInsertOverwrite(carbonLoadModel)
}
val done = CarbonLoaderUtil.recordNewLoadMetadata(metadataDetails, carbonLoadModel, false,
overwriteTable, uuid)
if (!done) {
val errorMessage = s"Dataload failed due to failure in table status updation for" +
s" ${carbonLoadModel.getTableName}"
LOGGER.error(errorMessage)
throw new Exception(errorMessage)
} else {
DataMapStatusManager.disableAllLazyDataMaps(carbonTable)
if (overwriteTable) {
val allDataMapSchemas = DataMapStoreManager.getInstance
.getDataMapSchemasOfTable(carbonTable).asScala
.filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
!dataMapSchema.isIndexDataMap).asJava
if (!allDataMapSchemas.isEmpty) {
DataMapStatusManager.truncateDataMap(allDataMapSchemas)
}
}
}
(done, metadataDetails)
}
/**
* Execute load process to load from input dataframe
*/
private def loadDataFrame(
sqlContext: SQLContext,
dataFrame: Option[DataFrame],
carbonLoadModel: CarbonLoadModel
): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
try {
val rdd = dataFrame.get.rdd
val nodeNumOfData = rdd.partitions.flatMap[String, Array[String]] { p =>
DataLoadPartitionCoalescer.getPreferredLocs(rdd, p).map(_.host)
}.distinct.length
val nodes = DistributionUtil.ensureExecutorsByNumberAndGetNodeList(
nodeNumOfData,
sqlContext.sparkContext)
val newRdd = new DataLoadCoalescedRDD[Row](sqlContext.sparkSession, rdd, nodes.toArray
.distinct)
new NewDataFrameLoaderRDD(
sqlContext.sparkSession,
new DataLoadResultImpl(),
carbonLoadModel,
newRdd
).collect()
} catch {
case ex: Exception =>
LOGGER.error("load data frame failed", ex)
throw ex
}
}
/**
* Execute load process to load from input file path specified in `carbonLoadModel`
*/
private def loadDataFile(
sqlContext: SQLContext,
carbonLoadModel: CarbonLoadModel,
hadoopConf: Configuration
): Array[(String, (LoadMetadataDetails, ExecutionErrors))] = {
/*
* when data load handle by node partition
* 1)clone the hadoop configuration,and set the file path to the configuration
* 2)use org.apache.hadoop.mapreduce.lib.input.TextInputFormat to get splits,size info
* 3)use CarbonLoaderUtil.nodeBlockMapping to get mapping info of node and block,
* for locally writing carbondata files(one file one block) in nodes