1
0

[HUDI-2101][RFC-28] support z-order for hudi (#3330)

* [HUDI-2101]support z-order for hudi

* Renaming some configs for consistency/simplicity.

* Minor code cleanups

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
xiarixiaoyao
2021-11-03 00:31:57 +08:00
committed by GitHub
parent f9bc3e03e5
commit d194643b49
22 changed files with 2140 additions and 10 deletions

View File

@@ -28,19 +28,20 @@ import org.apache.hudi.common.table.view.{FileSystemViewStorageConfig, HoodieTab
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
import org.apache.spark.api.java.JavaSparkContext
import org.apache.spark.internal.Logging
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.{Column, SparkSession}
import org.apache.spark.sql.avro.SchemaConverters
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BoundReference, Expression, InterpretedPredicate}
import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, InterpretedPredicate}
import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
import org.apache.spark.sql.catalyst.{InternalRow, expressions}
import org.apache.spark.sql.execution.datasources.{FileIndex, FileStatusCache, NoopCache, PartitionDirectory}
import org.apache.spark.sql.hudi.HoodieSqlUtils
import org.apache.spark.sql.hudi.{DataSkippingUtils, HoodieSqlUtils}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType
import org.apache.spark.unsafe.types.UTF8String
import java.util.Properties
import scala.collection.JavaConverters._
import scala.collection.JavaConversions._
import scala.collection.mutable
/**
@@ -84,6 +85,12 @@ case class HoodieFileIndex(
private val specifiedQueryInstant = options.get(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key)
.map(HoodieSqlUtils.formatQueryInstant)
/**
* Get all completeCommits.
*/
lazy val completedCommits = metaClient.getCommitsTimeline
.filterCompletedInstants().getInstants.iterator().toList.map(_.getTimestamp)
/**
* Get the schema of the table.
*/
@@ -147,6 +154,48 @@ case class HoodieFileIndex(
override def rootPaths: Seq[Path] = queryPath :: Nil
def enableDataSkipping(): Boolean = {
options.getOrElse(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(),
spark.sessionState.conf.getConfString(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "false")).toBoolean
}
private def filterFilesByDataSkippingIndex(dataFilters: Seq[Expression]): Set[String] = {
var allFiles: Set[String] = Set.empty
var candidateFiles: Set[String] = Set.empty
val indexPath = metaClient.getZindexPath
val fs = metaClient.getFs
if (fs.exists(new Path(indexPath)) && dataFilters.nonEmpty) {
// try to load latest index table from index path
val candidateIndexTables = fs.listStatus(new Path(indexPath)).filter(_.isDirectory)
.map(_.getPath.getName).filter(f => completedCommits.contains(f)).sortBy(x => x)
if (candidateIndexTables.nonEmpty) {
val dataFrameOpt = try {
Some(spark.read.load(new Path(indexPath, candidateIndexTables.last).toString))
} catch {
case _: Throwable =>
logError("missing index skip data-skipping")
None
}
if (dataFrameOpt.isDefined) {
val indexSchema = dataFrameOpt.get.schema
val indexFiles = DataSkippingUtils.getIndexFiles(spark.sparkContext.hadoopConfiguration, new Path(indexPath, candidateIndexTables.last).toString)
val indexFilter = dataFilters.map(DataSkippingUtils.createZindexFilter(_, indexSchema)).reduce(And)
logInfo(s"index filter condition: $indexFilter")
dataFrameOpt.get.persist()
if (indexFiles.size <= 4) {
allFiles = DataSkippingUtils.readParquetFile(spark, indexFiles)
} else {
allFiles = dataFrameOpt.get.select("file").collect().map(_.getString(0)).toSet
}
candidateFiles = dataFrameOpt.get.filter(new Column(indexFilter)).select("file").collect().map(_.getString(0)).toSet
dataFrameOpt.get.unpersist()
}
}
}
allFiles -- candidateFiles
}
/**
* Invoked by Spark to fetch list of latest base files per partition.
*
@@ -156,12 +205,29 @@ case class HoodieFileIndex(
*/
override def listFiles(partitionFilters: Seq[Expression],
dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {
// try to load filterFiles from index
val filterFiles: Set[String] = if (enableDataSkipping()) {
filterFilesByDataSkippingIndex(dataFilters)
} else {
Set.empty
}
if (queryAsNonePartitionedTable) { // Read as Non-Partitioned table.
Seq(PartitionDirectory(InternalRow.empty, allFiles))
val candidateFiles = if (!filterFiles.isEmpty) {
allFiles.filterNot(fileStatus => filterFiles.contains(fileStatus.getPath.getName))
} else {
allFiles
}
logInfo(s"Total files : ${allFiles.size}," +
s" candidate files after data skipping: ${candidateFiles.size} " +
s" skipping percent ${if (allFiles.length != 0) (allFiles.size - candidateFiles.size) / allFiles.size.toDouble else 0}")
Seq(PartitionDirectory(InternalRow.empty, candidateFiles))
} else {
// Prune the partition path by the partition filters
val prunedPartitions = prunePartition(cachedAllInputFileSlices.keys.toSeq, partitionFilters)
prunedPartitions.map { partition =>
var totalFileSize = 0
var candidateFileSize = 0
val result = prunedPartitions.map { partition =>
val baseFileStatuses = cachedAllInputFileSlices(partition).map(fileSlice => {
if (fileSlice.getBaseFile.isPresent) {
fileSlice.getBaseFile.get().getFileStatus
@@ -169,9 +235,19 @@ case class HoodieFileIndex(
null
}
}).filterNot(_ == null)
PartitionDirectory(partition.values, baseFileStatuses)
val candidateFiles = if (!filterFiles.isEmpty) {
baseFileStatuses.filterNot(fileStatus => filterFiles.contains(fileStatus.getPath.getName))
} else {
baseFileStatuses
}
totalFileSize += baseFileStatuses.size
candidateFileSize += candidateFiles.size
PartitionDirectory(partition.values, candidateFiles)
}
logInfo(s"Total files: ${totalFileSize}," +
s" Candidate files after data skipping : ${candidateFileSize} " +
s"skipping percent ${if (allFiles.length != 0) (totalFileSize - candidateFileSize) / totalFileSize.toDouble else 0}")
result
}
}

View File

@@ -0,0 +1,208 @@
/*
* 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.hudi
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.spark.sql.{AnalysisException, SparkSession}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, EqualNullSafe, EqualTo, Expression, ExtractValue, GetStructField, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, LessThan, LessThanOrEqual, Literal, Not, Or, StartsWith}
import org.apache.spark.sql.execution.datasources.PartitionedFile
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
import org.apache.spark.sql.functions.col
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.{StringType, StructType}
import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.unsafe.types.UTF8String
import scala.collection.JavaConverters._
object DataSkippingUtils {
/**
* create z_index filter and push those filters to index table to filter all candidate scan files.
* @param condition origin filter from query.
* @param indexSchema schema from index table.
* @return filters for index table.
*/
def createZindexFilter(condition: Expression, indexSchema: StructType): Expression = {
def buildExpressionInternal(colName: Seq[String], statisticValue: String): Expression = {
val appendColName = UnresolvedAttribute(colName).name + statisticValue
col(appendColName).expr
}
def reWriteCondition(colName: Seq[String], conditionExpress: Expression): Expression = {
val appendColName = UnresolvedAttribute(colName).name + "_minValue"
if (indexSchema.exists(p => p.name == appendColName)) {
conditionExpress
} else {
Literal.TrueLiteral
}
}
val minValue = (colName: Seq[String]) => buildExpressionInternal(colName, "_minValue")
val maxValue = (colName: Seq[String]) => buildExpressionInternal(colName, "_maxValue")
val num_nulls = (colName: Seq[String]) => buildExpressionInternal(colName, "_num_nulls")
condition match {
// query filter "colA = b" convert it to "colA_minValue <= b and colA_maxValue >= b" for index table
case EqualTo(attribute: AttributeReference, value: Literal) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, And(LessThanOrEqual(minValue(colName), value), GreaterThanOrEqual(maxValue(colName), value)))
// query filter "b = colA" convert it to "colA_minValue <= b and colA_maxValue >= b" for index table
case EqualTo(value: Literal, attribute: AttributeReference) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, And(LessThanOrEqual(minValue(colName), value), GreaterThanOrEqual(maxValue(colName), value)))
// query filter "colA = null" convert it to "colA_num_nulls = null" for index table
case equalNullSafe @ EqualNullSafe(_: AttributeReference, _ @ Literal(null, _)) =>
val colName = getTargetColNameParts(equalNullSafe.left)
reWriteCondition(colName, EqualTo(num_nulls(colName), equalNullSafe.right))
// query filter "colA < b" convert it to "colA_minValue < b" for index table
case LessThan(attribute: AttributeReference, value: Literal) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName,LessThan(minValue(colName), value))
// query filter "b < colA" convert it to "colA_maxValue > b" for index table
case LessThan(value: Literal, attribute: AttributeReference) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, GreaterThan(maxValue(colName), value))
// query filter "colA > b" convert it to "colA_maxValue > b" for index table
case GreaterThan(attribute: AttributeReference, value: Literal) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, GreaterThan(maxValue(colName), value))
// query filter "b > colA" convert it to "colA_minValue < b" for index table
case GreaterThan(value: Literal, attribute: AttributeReference) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, LessThan(minValue(colName), value))
// query filter "colA <= b" convert it to "colA_minValue <= b" for index table
case LessThanOrEqual(attribute: AttributeReference, value: Literal) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, LessThanOrEqual(minValue(colName), value))
// query filter "b <= colA" convert it to "colA_maxValue >= b" for index table
case LessThanOrEqual(value: Literal, attribute: AttributeReference) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, GreaterThanOrEqual(maxValue(colName), value))
// query filter "colA >= b" convert it to "colA_maxValue >= b" for index table
case GreaterThanOrEqual(attribute: AttributeReference, right: Literal) =>
val colName = getTargetColNameParts(attribute)
GreaterThanOrEqual(maxValue(colName), right)
// query filter "b >= colA" convert it to "colA_minValue <= b" for index table
case GreaterThanOrEqual(value: Literal, attribute: AttributeReference) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, LessThanOrEqual(minValue(colName), value))
// query filter "colA is null" convert it to "colA_num_nulls > 0" for index table
case IsNull(attribute: AttributeReference) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, GreaterThan(num_nulls(colName), Literal(0)))
// query filter "colA is not null" convert it to "colA_num_nulls = 0" for index table
case IsNotNull(attribute: AttributeReference) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, EqualTo(num_nulls(colName), Literal(0)))
// query filter "colA in (a,b)" convert it to " (colA_minValue <= a and colA_maxValue >= a) or (colA_minValue <= b and colA_maxValue >= b) " for index table
case In(attribute: AttributeReference, list: Seq[Literal]) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, list.map { lit =>
And(LessThanOrEqual(minValue(colName), lit), GreaterThanOrEqual(maxValue(colName), lit))
}.reduce(Or))
// query filter "colA like xxx" convert it to " (colA_minValue <= xxx and colA_maxValue >= xxx) or (colA_min start with xxx or colA_max start with xxx) " for index table
case StartsWith(attribute, v @ Literal(_: UTF8String, _)) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, Or(And(LessThanOrEqual(minValue(colName), v), GreaterThanOrEqual(maxValue(colName), v)) ,
Or(StartsWith(minValue(colName), v), StartsWith(maxValue(colName), v))))
// query filter "colA not in (a, b)" convert it to " (not( colA_minValue = a and colA_maxValue = a)) and (not( colA_minValue = b and colA_maxValue = b)) " for index table
case Not(In(attribute: AttributeReference, list: Seq[Literal])) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, list.map { lit =>
Not(And(EqualTo(minValue(colName), lit), EqualTo(maxValue(colName), lit)))
}.reduce(And))
// query filter "colA != b" convert it to "not ( colA_minValue = b and colA_maxValue = b )" for index table
case Not(EqualTo(attribute: AttributeReference, value: Literal)) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, Not(And(EqualTo(minValue(colName), value), EqualTo(maxValue(colName), value))))
// query filter "b != colA" convert it to "not ( colA_minValue = b and colA_maxValue = b )" for index table
case Not(EqualTo(value: Literal, attribute: AttributeReference)) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, Not(And(EqualTo(minValue(colName), value), EqualTo(maxValue(colName), value))))
// query filter "colA not like xxxx" convert it to "not ( colA_minValue startWith xxx and colA_maxValue startWith xxx)" for index table
case Not(StartsWith(attribute, value @ Literal(_: UTF8String, _))) =>
val colName = getTargetColNameParts(attribute)
reWriteCondition(colName, Not(And(StartsWith(minValue(colName), value), StartsWith(maxValue(colName), value))))
case or: Or =>
val resLeft = createZindexFilter(or.left, indexSchema)
val resRight = createZindexFilter(or.right, indexSchema)
Or(resLeft, resRight)
case and: And =>
val resLeft = createZindexFilter(and.left, indexSchema)
val resRight = createZindexFilter(and.right, indexSchema)
And(resLeft, resRight)
case expr: Expression =>
Literal.TrueLiteral
}
}
/**
* Extracts name from a resolved expression referring to a nested or non-nested column.
*/
def getTargetColNameParts(resolvedTargetCol: Expression): Seq[String] = {
resolvedTargetCol match {
case attr: Attribute => Seq(attr.name)
case Alias(c, _) => getTargetColNameParts(c)
case GetStructField(c, _, Some(name)) => getTargetColNameParts(c) :+ name
case ex: ExtractValue =>
throw new AnalysisException(s"convert reference to name failed, Updating nested fields is only supported for StructType: ${ex}.")
case other =>
throw new AnalysisException(s"convert reference to name failed, Found unsupported expression ${other}")
}
}
def getIndexFiles(conf: Configuration, indexPath: String): Seq[FileStatus] = {
val basePath = new Path(indexPath)
basePath.getFileSystem(conf)
.listStatus(basePath).filterNot(f => f.getPath.getName.endsWith(".parquet"))
}
/**
* read parquet files concurrently by local.
* this method is mush faster than spark
*/
def readParquetFile(spark: SparkSession, indexFiles: Seq[FileStatus], filters: Seq[Filter] = Nil, schemaOpts: Option[StructType] = None): Set[String] = {
val hadoopConf = spark.sparkContext.hadoopConfiguration
val partitionedFiles = indexFiles.map(f => PartitionedFile(InternalRow.empty, f.getPath.toString, 0, f.getLen))
val requiredSchema = new StructType().add("file", StringType, true)
val schema = schemaOpts.getOrElse(requiredSchema)
val parquetReader = new ParquetFileFormat().buildReaderWithPartitionValues(spark
, schema , StructType(Nil), requiredSchema, filters, Map.empty, hadoopConf)
val results = new Array[Iterator[String]](partitionedFiles.size)
partitionedFiles.zipWithIndex.par.foreach { case (pf, index) =>
val fileIterator = parquetReader(pf).asInstanceOf[Iterator[Any]]
val rows = fileIterator.flatMap(_ match {
case r: InternalRow => Seq(r)
case b: ColumnarBatch => b.rowIterator().asScala
}).map(r => r.getString(0))
results(index) = rows
}
results.flatMap(f => f).toSet
}
}

View File

@@ -0,0 +1,149 @@
/*
* 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.hudi.functional
import java.sql.{Date, Timestamp}
import org.apache.hadoop.fs.Path
import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig}
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions}
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
import org.apache.hudi.testutils.HoodieClientTestBase
import org.apache.spark.ZCurveOptimizeHelper
import org.apache.spark.sql._
import org.apache.spark.sql.types._
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import scala.collection.JavaConversions._
import scala.util.Random
class TestOptimizeTable extends HoodieClientTestBase {
var spark: SparkSession = null
val commonOpts = Map(
"hoodie.insert.shuffle.parallelism" -> "4",
"hoodie.upsert.shuffle.parallelism" -> "4",
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key() -> "partition",
DataSourceWriteOptions.PRECOMBINE_FIELD.key() -> "timestamp",
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
)
@BeforeEach override def setUp() {
initPath()
initSparkContexts()
spark = sqlContext.sparkSession
initTestDataGenerator()
initFileSystem()
}
@AfterEach override def tearDown() = {
cleanupSparkContexts()
cleanupTestDataGenerator()
cleanupFileSystem()
}
@ParameterizedTest
@ValueSource(strings = Array("COPY_ON_WRITE", "MERGE_ON_READ"))
def testOptimizewithClustering(tableType: String): Unit = {
// Bulk Insert Operation
val records1 = recordsToStrings(dataGen.generateInserts("001", 1000)).toList
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
inputDF1.write.format("org.apache.hudi")
.options(commonOpts)
.option("hoodie.compact.inline", "false")
.option(DataSourceWriteOptions.OPERATION.key(), DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.TABLE_TYPE.key(), tableType)
// option for clustering
.option("hoodie.parquet.small.file.limit", "0")
.option("hoodie.clustering.inline", "true")
.option("hoodie.clustering.inline.max.commits", "1")
.option("hoodie.clustering.plan.strategy.target.file.max.bytes", "1073741824")
.option("hoodie.clustering.plan.strategy.small.file.limit", "629145600")
.option("hoodie.clustering.plan.strategy.max.bytes.per.group", Long.MaxValue.toString)
.option("hoodie.clustering.plan.strategy.target.file.max.bytes", String.valueOf(64 *1024 * 1024L))
.option(HoodieClusteringConfig.LAYOUT_OPTIMIZE_ENABLE.key, "true")
.option(HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key, "begin_lat, begin_lon")
.mode(SaveMode.Overwrite)
.save(basePath)
assertEquals(1000, spark.read.format("hudi").load(basePath).count())
assertEquals(1000,
spark.read.option(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "true").format("hudi").load(basePath).count())
}
@Test
def testCollectMinMaxStatistics(): Unit = {
val testPath = new Path(System.getProperty("java.io.tmpdir"), "minMax")
val statisticPath = new Path(System.getProperty("java.io.tmpdir"), "stat")
val fs = testPath.getFileSystem(spark.sparkContext.hadoopConfiguration)
try {
val complexDataFrame = createComplexDataFrame(spark)
complexDataFrame.repartition(3).write.mode("overwrite").save(testPath.toString)
val df = spark.read.load(testPath.toString)
// do not support TimeStampType, so if we collect statistics for c4, should throw exception
val colDf = ZCurveOptimizeHelper.getMinMaxValue(df, "c1,c2,c3,c5,c6,c7,c8")
colDf.cache()
assertEquals(colDf.count(), 3)
assertEquals(colDf.take(1)(0).length, 22)
colDf.unpersist()
// try to save statistics
ZCurveOptimizeHelper.saveStatisticsInfo(df, "c1,c2,c3,c5,c6,c7,c8", statisticPath.toString, "2", Seq("0", "1"))
// save again
ZCurveOptimizeHelper.saveStatisticsInfo(df, "c1,c2,c3,c5,c6,c7,c8", statisticPath.toString, "3", Seq("0", "1", "2"))
// test old index table clean
ZCurveOptimizeHelper.saveStatisticsInfo(df, "c1,c2,c3,c5,c6,c7,c8", statisticPath.toString, "4", Seq("0", "1", "3"))
assertEquals(!fs.exists(new Path(statisticPath, "2")), true)
assertEquals(fs.exists(new Path(statisticPath, "3")), true)
} finally {
if (fs.exists(testPath)) fs.delete(testPath)
if (fs.exists(statisticPath)) fs.delete(statisticPath)
}
}
def createComplexDataFrame(spark: SparkSession): DataFrame = {
val schema = new StructType()
.add("c1", IntegerType)
.add("c2", StringType)
.add("c3", DecimalType(9,3))
.add("c4", TimestampType)
.add("c5", ShortType)
.add("c6", DateType)
.add("c7", BinaryType)
.add("c8", ByteType)
val rdd = spark.sparkContext.parallelize(0 to 1000, 1).map { item =>
val c1 = Integer.valueOf(item)
val c2 = s" ${item}sdc"
val c3 = new java.math.BigDecimal(s"${Random.nextInt(1000)}.${item}")
val c4 = new Timestamp(System.currentTimeMillis())
val c5 = java.lang.Short.valueOf(s"${(item + 16) /10}")
val c6 = Date.valueOf(s"${2020}-${item % 11 + 1}-${item % 28 + 1}")
val c7 = Array(item).map(_.toByte)
val c8 = java.lang.Byte.valueOf("9")
RowFactory.create(c1, c2, c3, c4, c5, c6, c7, c8)
}
spark.createDataFrame(rdd, schema)
}
}