@@ -27,7 +27,6 @@ import org.apache.spark.sql.execution.command.{DataLoadTableFileMapping, DataPro
27
27
import org .apache .spark .sql .hive .CarbonRelation
28
28
import org .apache .spark .util .{CausedBy , FileUtils }
29
29
30
- import org .apache .carbondata .common .constants .LoggerAction
31
30
import org .apache .carbondata .common .logging .{LogService , LogServiceFactory }
32
31
import org .apache .carbondata .core .constants .{CarbonCommonConstants , CarbonLoadOptionConstants }
33
32
import org .apache .carbondata .core .datastore .impl .FileFactory
@@ -39,14 +38,11 @@ import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
39
38
import org .apache .carbondata .core .util .path .CarbonStorePath
40
39
import org .apache .carbondata .format
41
40
import org .apache .carbondata .processing .exception .DataLoadingException
42
- import org .apache .carbondata .processing .loading .constants .DataLoadProcessorConstants
43
41
import org .apache .carbondata .processing .loading .exception .NoRetryException
44
42
import org .apache .carbondata .processing .loading .model .{CarbonDataLoadSchema , CarbonLoadModel }
45
- import org .apache .carbondata .processing .util .TableOptionConstant
46
43
import org .apache .carbondata .spark .exception .MalformedCarbonCommandException
47
- import org .apache .carbondata .spark .load .ValidateUtil
48
44
import org .apache .carbondata .spark .rdd .{CarbonDataRDDFactory , DictionaryLoadModel }
49
- import org .apache .carbondata .spark .util .{CommonUtil , GlobalDictionaryUtil }
45
+ import org .apache .carbondata .spark .util .{CommonUtil , DataLoadingUtil , GlobalDictionaryUtil }
50
46
51
47
case class LoadTableCommand (
52
48
databaseNameOp : Option [String ],
@@ -60,89 +56,6 @@ case class LoadTableCommand(
60
56
updateModel : Option [UpdateTableModel ] = None )
61
57
extends RunnableCommand with DataProcessCommand {
62
58
63
- private def getFinalOptions (carbonProperty : CarbonProperties ):
64
- scala.collection.mutable.Map [String , String ] = {
65
- val optionsFinal = scala.collection.mutable.Map [String , String ]()
66
- optionsFinal.put(" delimiter" , options.getOrElse(" delimiter" , " ," ))
67
- optionsFinal.put(" quotechar" , options.getOrElse(" quotechar" , " \" " ))
68
- optionsFinal.put(" fileheader" , options.getOrElse(" fileheader" , " " ))
69
- optionsFinal.put(" escapechar" , options.getOrElse(" escapechar" , " \\ " ))
70
- optionsFinal.put(" commentchar" , options.getOrElse(" commentchar" , " #" ))
71
- optionsFinal.put(" columndict" , options.getOrElse(" columndict" , null ))
72
- optionsFinal
73
- .put(" serialization_null_format" , options.getOrElse(" serialization_null_format" , " \\ N" ))
74
- optionsFinal.put(" bad_records_logger_enable" , options.getOrElse(" bad_records_logger_enable" ,
75
- carbonProperty
76
- .getProperty(CarbonLoadOptionConstants .CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE ,
77
- CarbonLoadOptionConstants .CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE_DEFAULT )))
78
- val badRecordActionValue = carbonProperty
79
- .getProperty(CarbonCommonConstants .CARBON_BAD_RECORDS_ACTION ,
80
- CarbonCommonConstants .CARBON_BAD_RECORDS_ACTION_DEFAULT )
81
- optionsFinal.put(" bad_records_action" , options.getOrElse(" bad_records_action" , carbonProperty
82
- .getProperty(CarbonLoadOptionConstants .CARBON_OPTIONS_BAD_RECORDS_ACTION ,
83
- badRecordActionValue)))
84
- optionsFinal
85
- .put(" is_empty_data_bad_record" , options.getOrElse(" is_empty_data_bad_record" , carbonProperty
86
- .getProperty(CarbonLoadOptionConstants .CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD ,
87
- CarbonLoadOptionConstants .CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD_DEFAULT )))
88
- optionsFinal.put(" all_dictionary_path" , options.getOrElse(" all_dictionary_path" , " " ))
89
- optionsFinal
90
- .put(" complex_delimiter_level_1" , options.getOrElse(" complex_delimiter_level_1" , " \\ $" ))
91
- optionsFinal
92
- .put(" complex_delimiter_level_2" , options.getOrElse(" complex_delimiter_level_2" , " \\ :" ))
93
- optionsFinal.put(" dateformat" , options.getOrElse(" dateformat" ,
94
- carbonProperty.getProperty(CarbonLoadOptionConstants .CARBON_OPTIONS_DATEFORMAT ,
95
- CarbonLoadOptionConstants .CARBON_OPTIONS_DATEFORMAT_DEFAULT )))
96
-
97
- optionsFinal.put(" global_sort_partitions" , options.getOrElse(" global_sort_partitions" ,
98
- carbonProperty
99
- .getProperty(CarbonLoadOptionConstants .CARBON_OPTIONS_GLOBAL_SORT_PARTITIONS , null )))
100
-
101
- optionsFinal.put(" maxcolumns" , options.getOrElse(" maxcolumns" , null ))
102
-
103
- optionsFinal.put(" batch_sort_size_inmb" , options.getOrElse(" batch_sort_size_inmb" ,
104
- carbonProperty.getProperty(CarbonLoadOptionConstants .CARBON_OPTIONS_BATCH_SORT_SIZE_INMB ,
105
- carbonProperty.getProperty(CarbonCommonConstants .LOAD_BATCH_SORT_SIZE_INMB ,
106
- CarbonCommonConstants .LOAD_BATCH_SORT_SIZE_INMB_DEFAULT ))))
107
-
108
- optionsFinal.put(" bad_record_path" , options.getOrElse(" bad_record_path" ,
109
- carbonProperty.getProperty(CarbonLoadOptionConstants .CARBON_OPTIONS_BAD_RECORD_PATH ,
110
- carbonProperty.getProperty(CarbonCommonConstants .CARBON_BADRECORDS_LOC ,
111
- CarbonCommonConstants .CARBON_BADRECORDS_LOC_DEFAULT_VAL ))))
112
-
113
- val useOnePass = options.getOrElse(" single_pass" ,
114
- carbonProperty.getProperty(CarbonLoadOptionConstants .CARBON_OPTIONS_SINGLE_PASS ,
115
- CarbonLoadOptionConstants .CARBON_OPTIONS_SINGLE_PASS_DEFAULT )).trim.toLowerCase match {
116
- case " true" =>
117
- true
118
- case " false" =>
119
- // when single_pass = false and if either alldictionarypath
120
- // or columnDict is configured the do not allow load
121
- if (StringUtils .isNotEmpty(optionsFinal(" all_dictionary_path" )) ||
122
- StringUtils .isNotEmpty(optionsFinal(" columndict" ))) {
123
- throw new MalformedCarbonCommandException (
124
- " Can not use all_dictionary_path or columndict without single_pass." )
125
- } else {
126
- false
127
- }
128
- case illegal =>
129
- val LOGGER = LogServiceFactory .getLogService(this .getClass.getCanonicalName)
130
- LOGGER .error(s " Can't use single_pass, because illegal syntax found: [ " + illegal + " ] " +
131
- " Please set it as 'true' or 'false'" )
132
- false
133
- }
134
- optionsFinal.put(" single_pass" , useOnePass.toString)
135
- optionsFinal
136
- }
137
-
138
- private def checkDefaultValue (value : String , default : String ) = {
139
- if (StringUtils .isEmpty(value)) {
140
- default
141
- } else {
142
- value
143
- }
144
- }
145
-
146
59
override def run (sparkSession : SparkSession ): Seq [Row ] = {
147
60
processData(sparkSession)
148
61
}
@@ -158,7 +71,6 @@ case class LoadTableCommand(
158
71
}
159
72
160
73
val dbName = databaseNameOp.getOrElse(sparkSession.catalog.currentDatabase)
161
-
162
74
val relation = CarbonEnv .getInstance(sparkSession).carbonMetastore
163
75
.lookupRelation(Option (dbName), tableName)(sparkSession).asInstanceOf [CarbonRelation ]
164
76
if (relation == null ) {
@@ -172,7 +84,7 @@ case class LoadTableCommand(
172
84
173
85
val carbonProperty : CarbonProperties = CarbonProperties .getInstance()
174
86
carbonProperty.addProperty(" zookeeper.enable.lock" , " false" )
175
- val optionsFinal = getFinalOptions (carbonProperty)
87
+ val optionsFinal = DataLoadingUtil .getDataLoadingOptions (carbonProperty, options )
176
88
177
89
val tableProperties = relation.tableMeta.carbonTable.getTableInfo
178
90
.getFactTable.getTableProperties
@@ -183,133 +95,26 @@ case class LoadTableCommand(
183
95
CarbonCommonConstants .LOAD_SORT_SCOPE_DEFAULT ))))
184
96
185
97
try {
98
+ val table = relation.tableMeta.carbonTable
99
+ val carbonLoadModel = new CarbonLoadModel ()
186
100
val factPath = if (dataFrame.isDefined) {
187
101
" "
188
102
} else {
189
103
FileUtils .getPaths(
190
104
CarbonUtil .checkAndAppendHDFSUrl(factPathFromUser))
191
105
}
192
- val carbonLoadModel = new CarbonLoadModel ()
193
- carbonLoadModel.setTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
194
- carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getDatabaseName)
195
- carbonLoadModel.setStorePath(relation.tableMeta.carbonTable.getStorePath)
196
-
197
- val table = relation.tableMeta.carbonTable
198
- carbonLoadModel.setTableName(table.getFactTableName)
199
- val dataLoadSchema = new CarbonDataLoadSchema (table)
200
- // Need to fill dimension relation
201
- carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
202
-
203
- val partitionLocation = relation.tableMeta.storePath + " /partition/" +
204
- relation.tableMeta.carbonTableIdentifier.getDatabaseName + " /" +
205
- relation.tableMeta.carbonTableIdentifier.getTableName + " /"
206
- val columnar = sparkSession.conf.get(" carbon.is.columnar.storage" , " true" ).toBoolean
207
- val sort_scope = optionsFinal(" sort_scope" )
208
- val single_pass = optionsFinal(" single_pass" )
209
- val bad_records_logger_enable = optionsFinal(" bad_records_logger_enable" )
210
- val bad_records_action = optionsFinal(" bad_records_action" )
211
- val bad_record_path = optionsFinal(" bad_record_path" )
212
- val global_sort_partitions = optionsFinal(" global_sort_partitions" )
213
- val dateFormat = optionsFinal(" dateformat" )
214
- val delimeter = optionsFinal(" delimiter" )
215
- val complex_delimeter_level1 = optionsFinal(" complex_delimiter_level_1" )
216
- val complex_delimeter_level2 = optionsFinal(" complex_delimiter_level_2" )
217
- val all_dictionary_path = optionsFinal(" all_dictionary_path" )
218
- val column_dict = optionsFinal(" columndict" )
219
- ValidateUtil .validateDateFormat(dateFormat, table, tableName)
220
- ValidateUtil .validateSortScope(table, sort_scope)
221
-
222
- if (bad_records_logger_enable.toBoolean ||
223
- LoggerAction .REDIRECT .name().equalsIgnoreCase(bad_records_action)) {
224
- if (! CarbonUtil .isValidBadStorePath(bad_record_path)) {
225
- sys.error(" Invalid bad records location." )
226
- }
227
- }
228
- carbonLoadModel.setBadRecordsLocation(bad_record_path)
229
-
230
- ValidateUtil .validateGlobalSortPartitions(global_sort_partitions)
231
- carbonLoadModel.setEscapeChar(checkDefaultValue(optionsFinal(" escapechar" ), " \\ " ))
232
- carbonLoadModel.setQuoteChar(checkDefaultValue(optionsFinal(" quotechar" ), " \" " ))
233
- carbonLoadModel.setCommentChar(checkDefaultValue(optionsFinal(" commentchar" ), " #" ))
234
-
235
- // if there isn't file header in csv file and load sql doesn't provide FILEHEADER option,
236
- // we should use table schema to generate file header.
237
- var fileHeader = optionsFinal(" fileheader" )
238
- val headerOption = options.get(" header" )
239
- if (headerOption.isDefined) {
240
- // whether the csv file has file header
241
- // the default value is true
242
- val header = try {
243
- headerOption.get.toBoolean
244
- } catch {
245
- case ex : IllegalArgumentException =>
246
- throw new MalformedCarbonCommandException (
247
- " 'header' option should be either 'true' or 'false'. " + ex.getMessage)
248
- }
249
- if (header) {
250
- if (fileHeader.nonEmpty) {
251
- throw new MalformedCarbonCommandException (
252
- " When 'header' option is true, 'fileheader' option is not required." )
253
- }
254
- } else {
255
- if (fileHeader.isEmpty) {
256
- fileHeader = table.getCreateOrderColumn(table.getFactTableName)
257
- .asScala.map(_.getColName).mkString(" ," )
258
- }
259
- }
260
- }
261
-
262
- carbonLoadModel.setDateFormat(dateFormat)
263
- carbonLoadModel.setDefaultTimestampFormat(carbonProperty.getProperty(
264
- CarbonCommonConstants .CARBON_TIMESTAMP_FORMAT ,
265
- CarbonCommonConstants .CARBON_TIMESTAMP_DEFAULT_FORMAT ))
266
- carbonLoadModel.setDefaultDateFormat(carbonProperty.getProperty(
267
- CarbonCommonConstants .CARBON_DATE_FORMAT ,
268
- CarbonCommonConstants .CARBON_DATE_DEFAULT_FORMAT ))
269
- carbonLoadModel
270
- .setSerializationNullFormat(
271
- TableOptionConstant .SERIALIZATION_NULL_FORMAT .getName + " ," +
272
- optionsFinal(" serialization_null_format" ))
273
- carbonLoadModel
274
- .setBadRecordsLoggerEnable(
275
- TableOptionConstant .BAD_RECORDS_LOGGER_ENABLE .getName + " ," + bad_records_logger_enable)
276
- carbonLoadModel
277
- .setBadRecordsAction(
278
- TableOptionConstant .BAD_RECORDS_ACTION .getName + " ," + bad_records_action)
279
- carbonLoadModel
280
- .setIsEmptyDataBadRecord(
281
- DataLoadProcessorConstants .IS_EMPTY_DATA_BAD_RECORD + " ," +
282
- optionsFinal(" is_empty_data_bad_record" ))
283
- carbonLoadModel.setSortScope(sort_scope)
284
- carbonLoadModel.setBatchSortSizeInMb(optionsFinal(" batch_sort_size_inmb" ))
285
- carbonLoadModel.setGlobalSortPartitions(global_sort_partitions)
286
- carbonLoadModel.setUseOnePass(single_pass.toBoolean)
287
- if (delimeter.equalsIgnoreCase(complex_delimeter_level1) ||
288
- complex_delimeter_level1.equalsIgnoreCase(complex_delimeter_level2) ||
289
- delimeter.equalsIgnoreCase(complex_delimeter_level2)) {
290
- sys.error(s " Field Delimiter & Complex types delimiter are same " )
291
- } else {
292
- carbonLoadModel.setComplexDelimiterLevel1(
293
- CarbonUtil .delimiterConverter(complex_delimeter_level1))
294
- carbonLoadModel.setComplexDelimiterLevel2(
295
- CarbonUtil .delimiterConverter(complex_delimeter_level2))
296
- }
297
- // set local dictionary path, and dictionary file extension
298
- carbonLoadModel.setAllDictPath(all_dictionary_path)
299
-
300
- val partitionStatus = CarbonCommonConstants .STORE_LOADSTATUS_SUCCESS
301
- try {
106
+ carbonLoadModel.setFactFilePath(factPath)
107
+ DataLoadingUtil .buildCarbonLoadModel(
108
+ table,
109
+ carbonProperty,
110
+ options,
111
+ optionsFinal,
112
+ carbonLoadModel
113
+ )
114
+
115
+ try {
302
116
// First system has to partition the data first and then call the load data
303
117
LOGGER .info(s " Initiating Direct Load for the Table : ( $dbName. $tableName) " )
304
- carbonLoadModel.setFactFilePath(factPath)
305
- carbonLoadModel.setCsvDelimiter(CarbonUtil .unescapeChar(delimeter))
306
- carbonLoadModel.setCsvHeader(fileHeader)
307
- carbonLoadModel.setColDictFilePath(column_dict)
308
- carbonLoadModel.setDirectLoad(true )
309
- carbonLoadModel.setCsvHeaderColumns(CommonUtil .getCsvHeaderColumns(carbonLoadModel))
310
- val validatedMaxColumns = CommonUtil .validateMaxColumns(carbonLoadModel.getCsvHeaderColumns,
311
- optionsFinal(" maxcolumns" ))
312
- carbonLoadModel.setMaxColumns(validatedMaxColumns.toString)
313
118
GlobalDictionaryUtil .updateTableMetadataFunc = updateTableMetadata
314
119
val storePath = relation.tableMeta.storePath
315
120
// add the start entry for the new load in the table status file
@@ -320,11 +125,9 @@ case class LoadTableCommand(
320
125
if (isOverwriteTable) {
321
126
LOGGER .info(s " Overwrite of carbon table with $dbName. $tableName is in progress " )
322
127
}
323
- if (null == carbonLoadModel.getLoadMetadataDetails) {
324
- CommonUtil .readLoadMetadataDetails(carbonLoadModel)
325
- }
326
128
if (carbonLoadModel.getLoadMetadataDetails.isEmpty && carbonLoadModel.getUseOnePass &&
327
- StringUtils .isEmpty(column_dict) && StringUtils .isEmpty(all_dictionary_path)) {
129
+ StringUtils .isEmpty(carbonLoadModel.getColDictFilePath) &&
130
+ StringUtils .isEmpty(carbonLoadModel.getAllDictPath)) {
328
131
LOGGER .info(s " Cannot use single_pass=true for $dbName. $tableName during the first load " )
329
132
LOGGER .audit(s " Cannot use single_pass=true for $dbName. $tableName during the first load " )
330
133
carbonLoadModel.setUseOnePass(false )
@@ -337,111 +140,21 @@ case class LoadTableCommand(
337
140
if (! FileFactory .isFileExist(metadataDirectoryPath, fileType)) {
338
141
FileFactory .mkdirs(metadataDirectoryPath, fileType)
339
142
}
143
+ val partitionStatus = CarbonCommonConstants .STORE_LOADSTATUS_SUCCESS
144
+ val columnar = sparkSession.conf.get(" carbon.is.columnar.storage" , " true" ).toBoolean
340
145
if (carbonLoadModel.getUseOnePass) {
341
- val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
342
- val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
343
- .getCarbonTableIdentifier
344
- val carbonTablePath = CarbonStorePath
345
- .getCarbonTablePath(storePath, carbonTableIdentifier)
346
- val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
347
- val dimensions = carbonTable.getDimensionByTableName(
348
- carbonTable.getFactTableName).asScala.toArray
349
- val colDictFilePath = carbonLoadModel.getColDictFilePath
350
- if (! StringUtils .isEmpty(colDictFilePath)) {
351
- carbonLoadModel.initPredefDictMap()
352
- // generate predefined dictionary
353
- GlobalDictionaryUtil
354
- .generatePredefinedColDictionary(colDictFilePath, carbonTableIdentifier,
355
- dimensions, carbonLoadModel, sparkSession.sqlContext, storePath, dictFolderPath)
356
- }
357
- if (! StringUtils .isEmpty(all_dictionary_path)) {
358
- carbonLoadModel.initPredefDictMap()
359
- GlobalDictionaryUtil
360
- .generateDictionaryFromDictionaryFiles(sparkSession.sqlContext,
361
- carbonLoadModel,
362
- storePath,
363
- carbonTableIdentifier,
364
- dictFolderPath,
365
- dimensions,
366
- all_dictionary_path)
367
- }
368
- // dictionaryServerClient dictionary generator
369
- val dictionaryServerPort = carbonProperty
370
- .getProperty(CarbonCommonConstants .DICTIONARY_SERVER_PORT ,
371
- CarbonCommonConstants .DICTIONARY_SERVER_PORT_DEFAULT )
372
- val sparkDriverHost = sparkSession.sqlContext.sparkContext.
373
- getConf.get(" spark.driver.host" )
374
- carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
375
- // start dictionary server when use one pass load and dimension with DICTIONARY
376
- // encoding is present.
377
- val allDimensions = table.getAllDimensions.asScala.toList
378
- val createDictionary = allDimensions.exists {
379
- carbonDimension => carbonDimension.hasEncoding(Encoding .DICTIONARY ) &&
380
- ! carbonDimension.hasEncoding(Encoding .DIRECT_DICTIONARY )
381
- }
382
- val server : Option [DictionaryServer ] = if (createDictionary) {
383
- val dictionaryServer = DictionaryServer
384
- .getInstance(dictionaryServerPort.toInt, carbonTable)
385
- carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
386
- sparkSession.sparkContext.addSparkListener(new SparkListener () {
387
- override def onApplicationEnd (applicationEnd : SparkListenerApplicationEnd ) {
388
- dictionaryServer.shutdown()
389
- }
390
- })
391
- Some (dictionaryServer)
392
- } else {
393
- None
394
- }
395
- CarbonDataRDDFactory .loadCarbonData(sparkSession.sqlContext,
146
+ loadDataUsingOnePass(
147
+ sparkSession,
148
+ carbonProperty,
396
149
carbonLoadModel,
397
- relation.tableMeta.storePath,
398
150
columnar,
399
- partitionStatus,
400
- server,
401
- isOverwriteTable,
402
- dataFrame,
403
- updateModel)
151
+ partitionStatus)
404
152
} else {
405
- val (dictionaryDataFrame, loadDataFrame) = if (updateModel.isDefined) {
406
- val fields = dataFrame.get.schema.fields
407
- import org .apache .spark .sql .functions .udf
408
- // extracting only segment from tupleId
409
- val getSegIdUDF = udf((tupleId : String ) =>
410
- CarbonUpdateUtil .getRequiredFieldFromTID(tupleId, TupleIdEnum .SEGMENT_ID ))
411
- // getting all fields except tupleId field as it is not required in the value
412
- var otherFields = fields.toSeq
413
- .filter(field => ! field.name
414
- .equalsIgnoreCase(CarbonCommonConstants .CARBON_IMPLICIT_COLUMN_TUPLEID ))
415
- .map(field => new Column (field.name))
416
-
417
- // extract tupleId field which will be used as a key
418
- val segIdColumn = getSegIdUDF(new Column (UnresolvedAttribute
419
- .quotedString(CarbonCommonConstants .CARBON_IMPLICIT_COLUMN_TUPLEID ))).
420
- as(CarbonCommonConstants .CARBON_IMPLICIT_COLUMN_SEGMENTID )
421
- // use dataFrameWithoutTupleId as dictionaryDataFrame
422
- val dataFrameWithoutTupleId = dataFrame.get.select(otherFields : _* )
423
- otherFields = otherFields :+ segIdColumn
424
- // use dataFrameWithTupleId as loadDataFrame
425
- val dataFrameWithTupleId = dataFrame.get.select(otherFields : _* )
426
- (Some (dataFrameWithoutTupleId), Some (dataFrameWithTupleId))
427
- } else {
428
- (dataFrame, dataFrame)
429
- }
430
-
431
- GlobalDictionaryUtil .generateGlobalDictionary(
432
- sparkSession.sqlContext,
433
- carbonLoadModel,
434
- relation.tableMeta.storePath,
435
- dictionaryDataFrame)
436
- CarbonDataRDDFactory .loadCarbonData(sparkSession.sqlContext,
153
+ loadData(
154
+ sparkSession,
437
155
carbonLoadModel,
438
- relation.tableMeta.storePath,
439
156
columnar,
440
- partitionStatus,
441
- None ,
442
- isOverwriteTable,
443
- loadDataFrame,
444
- updateModel)
157
+ partitionStatus)
445
158
}
446
159
} catch {
447
160
case CausedBy (ex : NoRetryException ) =>
@@ -454,6 +167,9 @@ case class LoadTableCommand(
454
167
} finally {
455
168
// Once the data load is successful delete the unwanted partition files
456
169
try {
170
+ val partitionLocation = table.getStorePath + " /partition/" +
171
+ table.getDatabaseName + " /" +
172
+ table.getFactTableName + " /"
457
173
val fileType = FileFactory .getFileType(partitionLocation)
458
174
if (FileFactory .isFileExist(partitionLocation, fileType)) {
459
175
val file = FileFactory
@@ -480,6 +196,130 @@ case class LoadTableCommand(
480
196
Seq .empty
481
197
}
482
198
199
+ private def loadDataUsingOnePass (
200
+ sparkSession : SparkSession ,
201
+ carbonProperty : CarbonProperties ,
202
+ carbonLoadModel : CarbonLoadModel ,
203
+ columnar : Boolean ,
204
+ partitionStatus : String ): Unit = {
205
+ val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
206
+ val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
207
+ .getCarbonTableIdentifier
208
+ val carbonTablePath = CarbonStorePath
209
+ .getCarbonTablePath(carbonLoadModel.getStorePath, carbonTableIdentifier)
210
+ val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
211
+ val dimensions = carbonTable.getDimensionByTableName(
212
+ carbonTable.getFactTableName).asScala.toArray
213
+ val colDictFilePath = carbonLoadModel.getColDictFilePath
214
+ if (! StringUtils .isEmpty(colDictFilePath)) {
215
+ carbonLoadModel.initPredefDictMap()
216
+ // generate predefined dictionary
217
+ GlobalDictionaryUtil .generatePredefinedColDictionary(
218
+ colDictFilePath,
219
+ carbonTableIdentifier,
220
+ dimensions,
221
+ carbonLoadModel,
222
+ sparkSession.sqlContext,
223
+ carbonLoadModel.getStorePath,
224
+ dictFolderPath)
225
+ }
226
+ if (! StringUtils .isEmpty(carbonLoadModel.getAllDictPath)) {
227
+ carbonLoadModel.initPredefDictMap()
228
+ GlobalDictionaryUtil
229
+ .generateDictionaryFromDictionaryFiles(sparkSession.sqlContext,
230
+ carbonLoadModel,
231
+ carbonLoadModel.getStorePath,
232
+ carbonTableIdentifier,
233
+ dictFolderPath,
234
+ dimensions,
235
+ carbonLoadModel.getAllDictPath)
236
+ }
237
+ // dictionaryServerClient dictionary generator
238
+ val dictionaryServerPort = carbonProperty
239
+ .getProperty(CarbonCommonConstants .DICTIONARY_SERVER_PORT ,
240
+ CarbonCommonConstants .DICTIONARY_SERVER_PORT_DEFAULT )
241
+ val sparkDriverHost = sparkSession.sqlContext.sparkContext.
242
+ getConf.get(" spark.driver.host" )
243
+ carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
244
+ // start dictionary server when use one pass load and dimension with DICTIONARY
245
+ // encoding is present.
246
+ val allDimensions =
247
+ carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.getAllDimensions.asScala.toList
248
+ val createDictionary = allDimensions.exists {
249
+ carbonDimension => carbonDimension.hasEncoding(Encoding .DICTIONARY ) &&
250
+ ! carbonDimension.hasEncoding(Encoding .DIRECT_DICTIONARY )
251
+ }
252
+ val server : Option [DictionaryServer ] = if (createDictionary) {
253
+ val dictionaryServer = DictionaryServer
254
+ .getInstance(dictionaryServerPort.toInt, carbonTable)
255
+ carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
256
+ sparkSession.sparkContext.addSparkListener(new SparkListener () {
257
+ override def onApplicationEnd (applicationEnd : SparkListenerApplicationEnd ) {
258
+ dictionaryServer.shutdown()
259
+ }
260
+ })
261
+ Some (dictionaryServer)
262
+ } else {
263
+ None
264
+ }
265
+ CarbonDataRDDFactory .loadCarbonData(sparkSession.sqlContext,
266
+ carbonLoadModel,
267
+ carbonLoadModel.getStorePath,
268
+ columnar,
269
+ partitionStatus,
270
+ server,
271
+ isOverwriteTable,
272
+ dataFrame,
273
+ updateModel)
274
+ }
275
+
276
+ private def loadData (
277
+ sparkSession : SparkSession ,
278
+ carbonLoadModel : CarbonLoadModel ,
279
+ columnar : Boolean ,
280
+ partitionStatus : String ): Unit = {
281
+ val (dictionaryDataFrame, loadDataFrame) = if (updateModel.isDefined) {
282
+ val fields = dataFrame.get.schema.fields
283
+ import org .apache .spark .sql .functions .udf
284
+ // extracting only segment from tupleId
285
+ val getSegIdUDF = udf((tupleId : String ) =>
286
+ CarbonUpdateUtil .getRequiredFieldFromTID(tupleId, TupleIdEnum .SEGMENT_ID ))
287
+ // getting all fields except tupleId field as it is not required in the value
288
+ var otherFields = fields.toSeq
289
+ .filter(field => ! field.name
290
+ .equalsIgnoreCase(CarbonCommonConstants .CARBON_IMPLICIT_COLUMN_TUPLEID ))
291
+ .map(field => new Column (field.name))
292
+
293
+ // extract tupleId field which will be used as a key
294
+ val segIdColumn = getSegIdUDF(new Column (UnresolvedAttribute
295
+ .quotedString(CarbonCommonConstants .CARBON_IMPLICIT_COLUMN_TUPLEID ))).
296
+ as(CarbonCommonConstants .CARBON_IMPLICIT_COLUMN_SEGMENTID )
297
+ // use dataFrameWithoutTupleId as dictionaryDataFrame
298
+ val dataFrameWithoutTupleId = dataFrame.get.select(otherFields : _* )
299
+ otherFields = otherFields :+ segIdColumn
300
+ // use dataFrameWithTupleId as loadDataFrame
301
+ val dataFrameWithTupleId = dataFrame.get.select(otherFields : _* )
302
+ (Some (dataFrameWithoutTupleId), Some (dataFrameWithTupleId))
303
+ } else {
304
+ (dataFrame, dataFrame)
305
+ }
306
+
307
+ GlobalDictionaryUtil .generateGlobalDictionary(
308
+ sparkSession.sqlContext,
309
+ carbonLoadModel,
310
+ carbonLoadModel.getStorePath,
311
+ dictionaryDataFrame)
312
+ CarbonDataRDDFactory .loadCarbonData(sparkSession.sqlContext,
313
+ carbonLoadModel,
314
+ carbonLoadModel.getStorePath,
315
+ columnar,
316
+ partitionStatus,
317
+ None ,
318
+ isOverwriteTable,
319
+ loadDataFrame,
320
+ updateModel)
321
+ }
322
+
483
323
private def updateTableMetadata (
484
324
carbonLoadModel : CarbonLoadModel ,
485
325
sqlContext : SQLContext ,
0 commit comments