-
Notifications
You must be signed in to change notification settings - Fork 703
/
CarbonLoadDataCommand.scala
405 lines (384 loc) · 17.9 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
/*
* 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 scala.collection.JavaConverters._
import org.apache.commons.lang3.StringUtils
import org.apache.hadoop.conf.Configuration
import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, UnresolvedAttribute}
import org.apache.spark.sql.execution.command.{DataCommand, DataLoadTableFileMapping, UpdateTableModel}
import org.apache.spark.sql.hive.CarbonRelation
import org.apache.spark.util.{CausedBy, FileUtils}
import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
import org.apache.carbondata.core.datastore.impl.FileFactory
import org.apache.carbondata.core.dictionary.server.DictionaryServer
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.CarbonDimension
import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, TupleIdEnum}
import org.apache.carbondata.core.statusmanager.SegmentStatus
import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
import org.apache.carbondata.core.util.path.CarbonStorePath
import org.apache.carbondata.events.{LoadTablePostExecutionEvent, LoadTablePreExecutionEvent, OperationContext, OperationListenerBus}
import org.apache.carbondata.format
import org.apache.carbondata.processing.exception.DataLoadingException
import org.apache.carbondata.processing.loading.TableProcessingOperations
import org.apache.carbondata.processing.loading.exception.NoRetryException
import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, DictionaryLoadModel}
import org.apache.carbondata.spark.util.{CommonUtil, DataLoadingUtil, 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,
dataFrame: Option[DataFrame] = None,
updateModel: Option[UpdateTableModel] = None,
var tableInfoOp: Option[TableInfo] = None,
internalOptions: Map[String, String] = Map.empty) extends DataCommand {
override def processData(sparkSession: SparkSession): Seq[Row] = {
val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
val carbonProperty: CarbonProperties = CarbonProperties.getInstance()
carbonProperty.addProperty("zookeeper.enable.lock", "false")
// 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 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
}
val tableProperties = table.getTableInfo.getFactTable.getTableProperties
val optionsFinal = DataLoadingUtil.getDataLoadingOptions(carbonProperty, options)
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))))
val factPath = if (dataFrame.isDefined) {
""
} else {
FileUtils.getPaths(
CarbonUtil.checkAndAppendHDFSUrl(factPathFromUser), hadoopConf)
}
carbonLoadModel.setFactFilePath(factPath)
carbonLoadModel.setAggLoadRequest(
internalOptions.getOrElse(CarbonCommonConstants.IS_INTERNAL_LOAD_CALL, "false").toBoolean)
carbonLoadModel.setSegmentId(internalOptions.getOrElse("mergedSegmentName", ""))
DataLoadingUtil.buildCarbonLoadModel(
table,
carbonProperty,
options,
optionsFinal,
carbonLoadModel,
hadoopConf
)
// 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)
try {
val operationContext = new OperationContext
val loadTablePreExecutionEvent: LoadTablePreExecutionEvent =
LoadTablePreExecutionEvent(sparkSession,
table.getCarbonTableIdentifier,
carbonLoadModel,
factPath,
dataFrame.isDefined,
optionsFinal,
options,
isOverwriteTable)
OperationListenerBus.getInstance.fireEvent(loadTablePreExecutionEvent, operationContext)
// 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)")
GlobalDictionaryUtil.updateTableMetadataFunc = updateTableMetadata
// add the start entry for the new load in the table status file
if (updateModel.isEmpty) {
CommonUtil.readAndUpdateLoadProgressInTableMeta(carbonLoadModel, isOverwriteTable)
}
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)
}
// Create table and metadata folders if not exist
val carbonTablePath = CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier)
val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
val fileType = FileFactory.getFileType(metadataDirectoryPath)
if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
FileFactory.mkdirs(metadataDirectoryPath, fileType)
}
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)
} else {
loadData(
sparkSession,
carbonLoadModel,
columnar,
partitionStatus,
hadoopConf,
operationContext)
}
val loadTablePostExecutionEvent: LoadTablePostExecutionEvent =
new LoadTablePostExecutionEvent(sparkSession,
table.getCarbonTableIdentifier,
carbonLoadModel)
OperationListenerBus.getInstance.fireEvent(loadTablePostExecutionEvent, operationContext)
} catch {
case CausedBy(ex: NoRetryException) =>
// update the load entry in table status file for changing the status to marked for delete
CommonUtil.updateTableStatusForFailure(carbonLoadModel)
LOGGER.error(ex, s"Dataload failure for $dbName.$tableName")
throw new RuntimeException(s"Dataload failure for $dbName.$tableName, ${ex.getMessage}")
case ex: Exception =>
// update the load entry in table status file for changing the status to marked for delete
CommonUtil.updateTableStatusForFailure(carbonLoadModel)
LOGGER.error(ex)
LOGGER.audit(s"Dataload failure for $dbName.$tableName. Please check the logs")
throw ex
} finally {
// 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 loadDataUsingOnePass(
sparkSession: SparkSession,
carbonProperty: CarbonProperties,
carbonLoadModel: CarbonLoadModel,
columnar: Boolean,
partitionStatus: SegmentStatus,
hadoopConf: Configuration,
operationContext: OperationContext): Unit = {
val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
.getCarbonTableIdentifier
val carbonTablePath = CarbonStorePath
.getCarbonTablePath(carbonLoadModel.getTablePath, carbonTableIdentifier)
val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
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)
// 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)
}
val server: Option[DictionaryServer] = if (createDictionary) {
val dictionaryServer = DictionaryServer
.getInstance(dictionaryServerPort.toInt, carbonTable)
carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
sparkSession.sparkContext.addSparkListener(new SparkListener() {
override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
dictionaryServer.shutdown()
}
})
Some(dictionaryServer)
} else {
None
}
CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
carbonLoadModel,
columnar,
partitionStatus,
server,
isOverwriteTable,
hadoopConf,
dataFrame,
updateModel,
operationContext)
}
private def loadData(
sparkSession: SparkSession,
carbonLoadModel: CarbonLoadModel,
columnar: Boolean,
partitionStatus: SegmentStatus,
hadoopConf: Configuration,
operationContext: OperationContext): Unit = {
val (dictionaryDataFrame, loadDataFrame) = if (updateModel.isDefined) {
val fields = dataFrame.get.schema.fields
import org.apache.spark.sql.functions.udf
// extracting only segment from tupleId
val getSegIdUDF = udf((tupleId: String) =>
CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.SEGMENT_ID))
// getting all fields except tupleId field as it is not required in the value
var otherFields = fields.toSeq.filter { field =>
!field.name.equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID)
}.map { field =>
new Column(field.name)
}
// extract tupleId field which will be used as a key
val segIdColumn = getSegIdUDF(new Column(UnresolvedAttribute
.quotedString(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))).
as(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_SEGMENTID)
// use dataFrameWithoutTupleId as dictionaryDataFrame
val dataFrameWithoutTupleId = dataFrame.get.select(otherFields: _*)
otherFields = otherFields :+ segIdColumn
// use dataFrameWithTupleId as loadDataFrame
val dataFrameWithTupleId = dataFrame.get.select(otherFields: _*)
(Some(dataFrameWithoutTupleId), Some(dataFrameWithTupleId))
} else {
(dataFrame, dataFrame)
}
if (!carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.isChildDataMap) {
GlobalDictionaryUtil.generateGlobalDictionary(
sparkSession.sqlContext,
carbonLoadModel,
hadoopConf,
dictionaryDataFrame)
}
CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
carbonLoadModel,
columnar,
partitionStatus,
None,
isOverwriteTable,
hadoopConf,
loadDataFrame,
updateModel,
operationContext)
}
private def updateTableMetadata(
carbonLoadModel: CarbonLoadModel,
sqlContext: SQLContext,
model: DictionaryLoadModel,
noDictDimension: Array[CarbonDimension]): Unit = {
val sparkSession = sqlContext.sparkSession
val carbonTablePath = CarbonStorePath.getCarbonTablePath(model.table)
val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
// read TableInfo
val tableInfo: format.TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
// modify TableInfo
val columns = tableInfo.getFact_table.getTable_columns
for (i <- 0 until columns.size) {
if (noDictDimension.exists(x => columns.get(i).getColumn_id.equals(x.getColumnId))) {
columns.get(i).encoders.remove(org.apache.carbondata.format.Encoding.DICTIONARY)
}
}
val entry = tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
entry.setTime_stamp(System.currentTimeMillis())
// write TableInfo
metastore.updateTableSchemaForAlter(carbonTablePath.getCarbonTableIdentifier,
carbonTablePath.getCarbonTableIdentifier,
tableInfo, entry, carbonTablePath.getPath)(sparkSession)
// update the schema modified time
metastore.updateAndTouchSchemasUpdatedTime()
val identifier = model.table.getCarbonTableIdentifier
// update CarbonDataLoadSchema
val carbonTable = metastore.lookupRelation(Option(identifier.getDatabaseName),
identifier.getTableName)(sqlContext.sparkSession).asInstanceOf[CarbonRelation].carbonTable
carbonLoadModel.setCarbonDataLoadSchema(new CarbonDataLoadSchema(carbonTable))
}
}