diff --git a/.travis.yml b/.travis.yml index 24e4a76fc4f..ffdd65cedd1 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,4 +1,6 @@ sudo: required dist: trusty language: java -script: mvn test +script: +- mvn clean -Pspark-2.1 test +- mvn clean -Pspark-2.2 test diff --git a/README.md b/README.md index 619a680fd84..a2fdfd457d7 100644 --- a/README.md +++ b/README.md @@ -3,11 +3,13 @@ OAP - Optimized Analytics Package (previously known as Spinach) is designed to accelerate Ad-hoc query. OAP defines a new parquet-like columnar storage data format and offering a fine-grained hierarchical cache mechanism in the unit of “Fiber” in memory. What’s more, OAP has extended the Spark SQL DDL to allow user to define the customized indices based on relation. ## Building +By defaut, it builds for Spark 2.1.0. To specify the Spark version, please use profile spark-2.1 or spark-2.2. ``` mvn -DskipTests package +mvn -DskipTests -Pspark-2.2 package ``` ## Prerequisites -You should have [Apache Spark](http://spark.apache.org/) of version 2.1.0 installed in your cluster. Refer to Apache Spark's [documents](http://spark.apache.org/docs/2.1.0/) for details. +You should have [Apache Spark](http://spark.apache.org/) of version 2.1.0 or 2.2.0 installed in your cluster. Refer to Apache Spark's [documents](http://spark.apache.org/docs/2.1.0/) for details. ## Use OAP with Spark 1. Build OAP, `mvn -DskipTests package` and find `oap-.jar` in `target/` 2. Deploy `oap-.jar` to master machine. diff --git a/pom.xml b/pom.xml index 337fd5033b3..f49713f039d 100644 --- a/pom.xml +++ b/pom.xml @@ -37,6 +37,7 @@ false ./ 9.2.16.v20160414 + 5.6.4 @@ -306,6 +307,34 @@ 2.13.0 test + + + org.elasticsearch + elasticsearch + ${elasticsearch.version} + + + + org.elasticsearch.client + transport + ${elasticsearch.version} + + + commons-logging + commons-logging + + + org.elasticsearch.plugin + transport-netty3-client + + + + + + org.elasticsearch.plugin + transport-netty4-client + ${elasticsearch.version} + @@ -446,9 +475,9 @@ -source - 1.7 + ${java.version} -target - 1.7 + ${java.version} -Xlint:all,-serial,-path @@ -498,8 +527,8 @@ maven-compiler-plugin 3.3 - 1.7 - 1.7 + ${java.version} + ${java.version} UTF-8 1024m true @@ -570,6 +599,7 @@ 2.1.0 2.11.8 2.11 + 1.7 @@ -595,6 +625,7 @@ src/main/spark2.2 + src/main/spark2.3 @@ -627,6 +658,7 @@ 2.2.0 2.11.8 2.11 + 1.8 @@ -652,6 +684,7 @@ src/main/spark2.1 + src/main/spark2.3 @@ -677,6 +710,65 @@ + + spark-2.3 + + 2.3.0 + 2.3.0 + 2.11.8 + 2.11 + 1.8 + + + + + org.antlr + antlr4-maven-plugin + 4.5.3 + + + + antlr4 + + + + + true + src/main/spark2.3/antlr4 + + + + org.apache.maven.plugins + maven-compiler-plugin + + + src/main/spark2.1 + src/main/spark2.2 + + + + + org.codehaus.mojo + build-helper-maven-plugin + 3.0.0 + + + add-source + generate-sources + + add-source + + + + src/main/spark2.3 + + + + + + + + diff --git a/src/main/java/org/apache/spark/sql/execution/datasources/parquet/OapTransportClient.java b/src/main/java/org/apache/spark/sql/execution/datasources/parquet/OapTransportClient.java new file mode 100644 index 00000000000..fa7ec5b181b --- /dev/null +++ b/src/main/java/org/apache/spark/sql/execution/datasources/parquet/OapTransportClient.java @@ -0,0 +1,113 @@ +/* + * 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.datasources.parquet; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import org.elasticsearch.client.transport.TransportClient; +import org.elasticsearch.common.network.NetworkModule; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.reindex.ReindexPlugin; +import org.elasticsearch.join.ParentJoinPlugin; +import org.elasticsearch.percolator.PercolatorPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.script.mustache.MustachePlugin; +import org.elasticsearch.transport.Netty4Plugin; + +import io.netty.util.ThreadDeathWatcher; +import io.netty.util.concurrent.GlobalEventExecutor; + +public class OapTransportClient extends TransportClient { + + /** + * Netty wants to do some unwelcome things like use unsafe and replace a private field, or use + * a poorly considered buffer recycler. This method disables these things by default, but can + * be overridden by setting the corresponding system properties. + */ + private static void initializeNetty() { + /* + * We disable three pieces of Netty functionality here: + * - we disable Netty from being unsafe + * - we disable Netty from replacing the selector key set + * - we disable Netty from using the recycler + * + * While permissions are needed to read and set these, the permissions needed here are + * innocuous and thus should simply be granted rather than us handling a security + * exception here. + */ + setSystemPropertyIfUnset("io.netty.noUnsafe", Boolean.toString(true)); + setSystemPropertyIfUnset("io.netty.noKeySetOptimization", Boolean.toString(true)); + setSystemPropertyIfUnset("io.netty.recycler.maxCapacityPerThread", Integer.toString(0)); + } + + private static void setSystemPropertyIfUnset(final String key, final String value) { + final String currentValue = System.getProperty(key); + if (currentValue == null) { + System.setProperty(key, value); + } + } + + private static final List OPTIONAL_DEPENDENCIES = Arrays.asList( // + "org.elasticsearch.transport.Netty3Plugin", // + "org.elasticsearch.transport.Netty4Plugin"); + + private static final Collection> PRE_INSTALLED_PLUGINS; + + static { + + initializeNetty(); + + List> plugins = new ArrayList<>(); + + plugins.add(Netty4Plugin.class); + plugins.add(ReindexPlugin.class); + plugins.add(PercolatorPlugin.class); + plugins.add(MustachePlugin.class); + plugins.add(ParentJoinPlugin.class); + + PRE_INSTALLED_PLUGINS = Collections.unmodifiableList(plugins); + } + + public OapTransportClient(Settings settings) { + super(settings, PRE_INSTALLED_PLUGINS); + } + + @Override + public void close() { + super.close(); + if (NetworkModule.TRANSPORT_TYPE_SETTING.exists(settings) == false + || NetworkModule.TRANSPORT_TYPE_SETTING.get(settings).equals( + Netty4Plugin.NETTY_TRANSPORT_NAME)) { + try { + GlobalEventExecutor.INSTANCE.awaitInactivity(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + try { + ThreadDeathWatcher.awaitInactivity(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } +} diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/DataSourceMeta.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/DataSourceMeta.scala index 051dccc4154..c2003fdc6ac 100644 --- a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/DataSourceMeta.scala +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/DataSourceMeta.scala @@ -134,6 +134,12 @@ private[oap] case class BitMapIndex(entries: Seq[Int] = Nil) extends IndexType { override def toString: String = "COLUMN(" + entries.mkString(", ") + ") BITMAP" } +private[oap] case class ESIndex(entries: Seq[Int] = Nil) extends IndexType { + def appendEntry(entry: Int): ESIndex = ESIndex(entries :+ entry) + + override def toString: String = "COLUMN(" + entries.mkString(", ") + ") ES" +} + private[oap] case class HashIndex(entries: Seq[Int] = Nil) extends IndexType { def appendEntry(entry: Int): HashIndex = HashIndex(entries :+ entry) @@ -237,6 +243,10 @@ private[oap] class IndexMeta( out.writeByte(HASH_INDEX_TYPE) entries.foreach(keyBits += _) writeBitSet(keyBits, INDEX_META_KEY_LENGTH, out) + case ESIndex(entries) => + out.writeByte(ES_INDEX_TYPE) + entries.foreach(keyBits += _) + writeBitSet(keyBits, INDEX_META_KEY_LENGTH, out) } } @@ -268,6 +278,7 @@ private[oap] class IndexMeta( flag match { case BITMAP_INDEX_TYPE => BitMapIndex(keyBits.toSeq) case HASH_INDEX_TYPE => HashIndex(keyBits.toSeq) + case ES_INDEX_TYPE => ESIndex(keyBits.toSeq) } } } @@ -277,6 +288,7 @@ private[oap] object IndexMeta { final val BTREE_INDEX_TYPE = 0 final val BITMAP_INDEX_TYPE = 1 final val HASH_INDEX_TYPE = 2 + final val ES_INDEX_TYPE = 3 def apply(): IndexMeta = new IndexMeta() def apply(name: String, time: String, indexType: IndexType): IndexMeta = { @@ -356,6 +368,9 @@ private[oap] case class DataSourceMeta( case index @ BitMapIndex(entries) => entries.map(ordinal => schema(ordinal).name).contains(attr) && index.satisfy(requirement) + case index @ ESIndex(entries) => + entries.map(ordinal => + schema(ordinal).name).contains(attr) && index.satisfy(requirement) case _ => false } } diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/OapFileFormat.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/OapFileFormat.scala index 472212783dd..3fd98509dc5 100644 --- a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/OapFileFormat.scala +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/OapFileFormat.scala @@ -282,7 +282,6 @@ private[sql] class OapFileFormat extends FileFormat throw new OapException("Unexpected data file version") Iterator.empty } - } case None => (_: PartitionedFile) => { // TODO need to think about when there is no oap.meta file at all @@ -428,45 +427,6 @@ private[oap] case class OapWriteResult( rowsWritten: Int, partitionString: String) -private[oap] class OapOutputWriter( - path: String, - dataSchema: StructType, - context: TaskAttemptContext) extends OutputWriter { - private var rowCount = 0 - private var partitionString: String = "" - - override def setPartitionString(ps: String): Unit = { - partitionString = ps - } - - private val writer: OapDataWriter = { - val isCompressed = FileOutputFormat.getCompressOutput(context) - val conf = context.getConfiguration - val file: Path = new Path(path) - val fs = file.getFileSystem(conf) - val fileOut = fs.create(file, false) - - new OapDataWriter(isCompressed, fileOut, dataSchema, conf) - } - - override def write(row: Row): Unit = throw new NotImplementedError("write(row: Row)") - - override protected[sql] def writeInternal(row: InternalRow): Unit = { - rowCount += 1 - writer.write(row) - } - - override def close(): Unit = { - writer.close() - } - - override def writeStatus(): WriteResult = { - OapWriteResult(dataFileName, rowCount, partitionString) - } - - def dataFileName: String = new Path(path).getName -} - private[sql] object OapFileFormat { val OAP_DATA_EXTENSION = ".data" val OAP_INDEX_EXTENSION = ".index" diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/OapStrategies.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/OapStrategies.scala index 8ade55b816a..f233fca59de 100644 --- a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/OapStrategies.scala +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/OapStrategies.scala @@ -19,21 +19,25 @@ package org.apache.spark.sql.execution.datasources.oap import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{execution, SparkSession, Strategy} -import org.apache.spark.sql.catalyst.{expressions, InternalRow} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.Strategy +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalAggregation, PhysicalOperation} import org.apache.spark.sql.catalyst.plans.{logical, LeftSemi} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.execution import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.aggregate.OapAggUtils import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.oap.utils.CaseInsensitiveMap import org.apache.spark.sql.execution.joins.BuildRight import org.apache.spark.sql.internal.oap.OapConf +import org.apache.spark.sql.oap.adapter.{AggregateFunctionAdapter, FileIndexAdapter, FileSourceScanExecAdapter, LogicalPlanAdapter} import org.apache.spark.util.Utils trait OapStrategies extends Logging { @@ -104,7 +108,7 @@ trait OapStrategies extends Logging { val filterAttributes = AttributeSet(ExpressionSet(filters)) val orderAttributes = AttributeSet(ExpressionSet(order.map(_.child))) if (orderAttributes.size == 1 && filterAttributes == orderAttributes) { - val oapOption = new CaseInsensitiveMap(file.options + + val oapOption = CaseInsensitiveMap(file.options + (OapFileFormat.OAP_QUERY_LIMIT_OPTION_KEY -> limit.toString) + (OapFileFormat.OAP_QUERY_ORDER_OPTION_KEY -> order.head.isAscending.toString)) val indexRequirement = filters.map(_ => BTreeIndex()) @@ -147,8 +151,8 @@ trait OapStrategies extends Logging { // We can take a much larger threshold here since if this optimization // is applicable, only distinct item will be broadcasted, those data // should much less than the origin table. - plan.statistics.isBroadcastable || - plan.statistics.sizeInBytes <= conf.autoBroadcastJoinThreshold + val stats = LogicalPlanAdapter.getStatistics(plan, conf) + stats.sizeInBytes <= conf.autoBroadcastJoinThreshold } def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { @@ -169,7 +173,7 @@ trait OapStrategies extends Logging { val filterAttributes = AttributeSet(ExpressionSet(filters)) val orderAttributes = AttributeSet(ExpressionSet(order.map(_.child))) if (orderAttributes.size == 1 || filterAttributes == orderAttributes) { - val oapOption = new CaseInsensitiveMap(file.options + + val oapOption = CaseInsensitiveMap(file.options + (OapFileFormat.OAP_INDEX_SCAN_NUM_OPTION_KEY -> "1")) val indexRequirement = filters.map(_ => BitMapIndex()) @@ -214,7 +218,8 @@ trait OapStrategies extends Logging { } val aggregateOperator = - if (aggregateExpressions.map(_.aggregateFunction).exists(!_.supportsPartial)) { + if (aggregateExpressions.map(_.aggregateFunction).exists( + !AggregateFunctionAdapter.supportsPartial(_))) { if (functionsWithDistinct.nonEmpty) { sys.error("Distinct columns cannot exist in Aggregate operator containing " + "aggregate functions which don't support partial aggregation.") @@ -253,7 +258,7 @@ trait OapStrategies extends Logging { val indexRequirement = filters.map(_ => BTreeIndex()) if (groupingAttributes.size == 1 && filterAttributes == groupingAttributes) { - val oapOption = new CaseInsensitiveMap(file.options + + val oapOption = CaseInsensitiveMap(file.options + (OapFileFormat.OAP_INDEX_GROUP_BY_OPTION_KEY -> "true")) createOapFileScanPlan( @@ -331,7 +336,8 @@ trait OapStrategies extends Logging { ExpressionSet(normalizedFilters.filter(_.references.subsetOf(partitionSet))) logInfo(s"Pruning directories with: ${partitionKeyFilters.mkString(",")}") - val selectedPartitions = _fsRelation.location.listFiles(partitionKeyFilters.toSeq) + val selectedPartitions = FileIndexAdapter.listFiles( + _fsRelation.location, partitionKeyFilters.toSeq, Nil) _fsRelation.fileFormat match { case fileFormat: OapFileFormat => @@ -360,20 +366,16 @@ trait OapStrategies extends Logging { val outputSchema = readDataColumns.toStructType logInfo(s"Output Data Schema: ${outputSchema.simpleString(5)}") - val pushedDownFilters = dataFilters.flatMap(DataSourceStrategy.translateFilter) - logInfo(s"Pushed Filters: ${pushedDownFilters.mkString(",")}") - val outputAttributes = readDataColumns ++ partitionColumns val oapRelation = _fsRelation.copy(options = oapOption)(_fsRelation.sparkSession) - val scan = - new FileSourceScanExec( - oapRelation, - outputAttributes, - outputSchema, - partitionKeyFilters.toSeq, - pushedDownFilters, - table.map(_.identifier)) + val scan = FileSourceScanExecAdapter.createFileSourceScanExec( + oapRelation, + outputAttributes, + outputSchema, + partitionKeyFilters.toSeq, + dataFilters, + table.map(_.identifier)) val afterScanFilter = afterScanFilters.toSeq.reduceOption(expressions.And) val withFilter = afterScanFilter.map(execution.FilterExec(_, scan)).getOrElse(scan) diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/filecache/FiberSensor.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/filecache/FiberSensor.scala index 96b6c8a2bfe..45353278faf 100644 --- a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/filecache/FiberSensor.scala +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/filecache/FiberSensor.scala @@ -19,12 +19,9 @@ package org.apache.spark.sql.execution.datasources.oap.filecache import java.util.concurrent.ConcurrentHashMap -import scala.collection.JavaConverters._ - import com.google.common.base.Throwables import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.datasources.oap.IndexMeta import org.apache.spark.sql.execution.datasources.oap.utils.CacheStatusSerDe import org.apache.spark.sql.oap.listener.SparkListenerCustomInfoUpdate import org.apache.spark.util.collection.BitSet @@ -37,26 +34,19 @@ private[oap] case class FiberCacheStatus( val cachedFiberCount = bitmask.cardinality() - def moreCacheThan(other: FiberCacheStatus): Boolean = { - if (cachedFiberCount >= other.cachedFiberCount) { - true - } else { - false - } - } + def moreCacheThan(other: FiberCacheStatus): Boolean = cachedFiberCount >= other.cachedFiberCount } -private[oap] case class IndexFiberCacheStatus(file: String, meta: IndexMeta) +// TODO: FiberSensor doesn't consider the fiber cache, but only the number of cached fiber count +// FiberSensor is the FiberCache info recorder on Driver, it contains a file cache location mapping +// (for cache locality) and metrics info +private[sql] class FiberSensor extends Logging { - -// TODO FiberSensor doesn't consider the fiber cache, but only the number of cached -// fiber count -private[oap] trait AbstractFiberSensor extends Logging { - case class HostFiberCache(host: String, status: FiberCacheStatus) + private case class HostFiberCache(host: String, status: FiberCacheStatus) private val fileToHost = new ConcurrentHashMap[String, HostFiberCache] - def update(fiberInfo: SparkListenerCustomInfoUpdate): Unit = { + def updateLocations(fiberInfo: SparkListenerCustomInfoUpdate): Unit = { val updateExecId = fiberInfo.executorId val updateHostName = fiberInfo.hostName val host = FiberSensor.OAP_CACHE_HOST_PREFIX + updateHostName + @@ -76,16 +66,34 @@ private[oap] trait AbstractFiberSensor extends Logging { } } + // TODO: define a function to wrap this and make it private + private[sql] val executorToCacheManager = new ConcurrentHashMap[String, CacheStats]() + + def updateMetrics(fiberInfo: SparkListenerCustomInfoUpdate): Unit = { + if (fiberInfo.customizedInfo.nonEmpty) { + try { + val cacheMetrics = CacheStats(fiberInfo.customizedInfo) + executorToCacheManager.put(fiberInfo.executorId, cacheMetrics) + logDebug(s"execID:${fiberInfo.executorId}, host:${fiberInfo.hostName}," + + s" ${cacheMetrics.toDebugString}") + } catch { + case t: Throwable => + val stack = Throwables.getStackTraceAsString(t) + logError(s"FiberSensor parse json failed, $stack") + } + } + } + /** * get hosts that has fiber cached for fiber file. * Current implementation only returns one host, but still using API name with [[getHosts]] * @param filePath fiber file's path * @return */ - def getHosts(filePath: String): Option[String] = { + def getHosts(filePath: String): Seq[String] = { fileToHost.get(filePath) match { - case HostFiberCache(host, status) => Some(host) - case null => None + case HostFiberCache(host, status) => Seq(host) + case null => Seq.empty } } } @@ -94,27 +102,3 @@ private[oap] object FiberSensor { val OAP_CACHE_HOST_PREFIX = "OAP_HOST_" val OAP_CACHE_EXECUTOR_PREFIX = "_OAP_EXECUTOR_" } - -private[sql] class FiberSensor extends AbstractFiberSensor - -object FiberCacheManagerSensor extends AbstractFiberSensor { - val executorToCacheManager = new ConcurrentHashMap[String, CacheStats]() - - def summary(): CacheStats = executorToCacheManager.asScala.toMap.values - .foldLeft(CacheStats())((sum, cache) => sum + cache) - - override def update(fiberInfo: SparkListenerCustomInfoUpdate): Unit = { - if (fiberInfo.customizedInfo.nonEmpty) { - try { - val cacheMetrics = CacheStats(fiberInfo.customizedInfo) - executorToCacheManager.put(fiberInfo.executorId, cacheMetrics) - logDebug(s"execID:${fiberInfo.executorId}, host:${fiberInfo.hostName}," + - s" ${cacheMetrics.toDebugString}") - } catch { - case t: Throwable => - val stack = Throwables.getStackTraceAsString(t) - logError(s"FiberCacheManagerSensor parse json failed, $stack") - } - } - } -} diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/IndexContext.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/IndexContext.scala index 3c2cf4fd7a4..9af3258f34e 100644 --- a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/IndexContext.scala +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/IndexContext.scala @@ -19,13 +19,13 @@ package org.apache.spark.sql.execution.datasources.oap.index import scala.collection.mutable import scala.collection.mutable.ArrayBuffer - import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.JoinedRow import org.apache.spark.sql.catalyst.expressions.codegen.GenerateOrdering import org.apache.spark.sql.execution.datasources.oap._ import org.apache.spark.sql.execution.datasources.oap.index.ScannerBuilder.IntervalArrayMap +import org.apache.spark.sql.execution.datasources.oap.index.elasticsearch.ESScanner import org.apache.spark.sql.types.StructType private[oap] class IndexContext(meta: DataSourceMeta) extends Logging { @@ -57,8 +57,8 @@ private[oap] class IndexContext(meta: DataSourceMeta) extends Logging { logDebug("Selecting Available Index:") val indexDisableList = indexDisableListStr.split(",").map(_.trim).toSeq - meta.indexMetas.filterNot(meta => indexDisableList.contains(meta.name)).foreach { - indexMeta => indexMeta.indexType match { + meta.indexMetas.filterNot(meta => indexDisableList.contains(meta.name)).foreach { indexMeta => + indexMeta.indexType match { case BTreeIndex(entries) if entries.length == 1 => val attribute = meta.schema(entries(0).ordinal).name if (intervalMap.contains(attribute)) { @@ -99,6 +99,12 @@ private[oap] class IndexContext(meta: DataSourceMeta) extends Logging { availableIndexes.append((entries.indexOf(entry), indexMeta)) } } + case ESIndex(entries) => + for (entry <- entries) { + if (intervalMap.contains(meta.schema(entry).name)) { + availableIndexes.append((entries.indexOf(entry), indexMeta)) + } + } case other => // TODO support other types of index } } @@ -143,6 +149,8 @@ private[oap] class IndexContext(meta: DataSourceMeta) extends Logging { entries.map(entry => meta.schema(entry.ordinal).name) case BitMapIndex(entries) => entries.map(entry => meta.schema(entry).name) + case ESIndex(entries) => + entries.map(entry => meta.schema(entry).name) case _ => Seq.empty } } @@ -293,6 +301,14 @@ private[oap] class IndexContext(meta: DataSourceMeta) extends Logging { logDebug("Bitmap index only supports equal query.") scanner.intervalArray = singleValueIntervalArray } + case ESIndex(entries) => + val attribute = meta.schema(entries(lastIdx)).name + val likeIntervals = intervalMap(attribute).filter(_.isPrefixMatch) + if (likeIntervals.length == 1) { + keySchema = new StructType().add(meta.schema(entries(lastIdx))) + scanner = ESScanner(bestIndexer) + scanner.intervalArray = likeIntervals + } case _ => } @@ -494,8 +510,7 @@ private[oap] class FilterOptimizer(keySchema: StructType) { interval1 <- intervalArray1 interval2 <- intervalArray2 // isNull & otherPredicate => empty - if (!(interval1.isNullPredicate ^ interval2.isNullPredicate)) && - !interval1.isPrefixMatch && !interval2.isPrefixMatch + if (!(interval1.isNullPredicate ^ interval2.isNullPredicate)) } yield { // isNull & isNull => isNull if (interval1.isNullPredicate && interval2.isNullPredicate) { @@ -517,6 +532,7 @@ private[oap] class FilterOptimizer(keySchema: StructType) { interval1.endInclude, interval2.endInclude, isEndKey = true) interval.end = re2._1 interval.endInclude = re2._2 + interval.isPrefixMatch = interval1.isPrefixMatch || interval2.isPrefixMatch interval } } diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/IndexScanner.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/IndexScanner.scala index 2f4a5bc5c90..20ea43e124c 100644 --- a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/IndexScanner.scala +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/IndexScanner.scala @@ -322,7 +322,6 @@ private[oap] object ScannerBuilder extends Logging { filters.filterNot(canSupport(_, ic)) } - } private[oap] class IndexScanners(val scanners: Seq[IndexScanner]) diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapEsIndex.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapEsIndex.scala new file mode 100644 index 00000000000..c99c969d422 --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapEsIndex.scala @@ -0,0 +1,258 @@ +/* + * 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.datasources.oap.index + +import java.net.InetAddress + +import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest +import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsRequest +import org.elasticsearch.action.admin.indices.refresh.RefreshRequest +import org.elasticsearch.client.transport.TransportClient +import org.elasticsearch.common.settings.Settings +import org.elasticsearch.common.transport.InetSocketTransportAddress +import org.elasticsearch.common.xcontent.XContentFactory +import org.elasticsearch.index.query.QueryBuilders + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.datasources.oap.Key +import org.apache.spark.sql.execution.datasources.parquet.OapTransportClient + +/* +class OapEsIndex(host: String, port: Int) extends OapEsUtil with Logging { + + // We only use one document type in ES. + private val docType = "oap_records" + + private val client = getEsClient + + private val mappings = { + XContentFactory.jsonBuilder() + .startObject() + // Settings of this Index + .startObject("settings") + .field("number_of_shards", 1) + .field("number_of_replicas", 0) + .endObject() + .startObject("mappings") + // This is the document type, called "oap_records" + .startObject("oap_records") + .startObject("properties") + // value in indexed column. + .startObject("key") + .field("type", "text") + .field("store", "yes") + .field("index", "analyzed") + // TODO: Remove this in production + .field("fielddata", "true") + .endObject() + // index file name = data file name + index name + .startObject("index_file_name") + .field("type", "string") + .field("store", "yes") + .field("index", "not_analyzed") + .endObject() + // The row id of this record in data file + .startObject("rowId") + .field("type", "integer") + .field("store", "yes") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + } + + /** + * Drop the whole table index. Usually, we don't need this. + */ + def dropESTableIndex(tableIndexName: String): Unit = { + + if (checkEsTableIndexExists(tableIndexName)) { + + client.admin().indices() + .delete(new DeleteIndexRequest(tableIndexName)) + .actionGet() + } else { + + logWarning(s"ES Index: $tableIndexName is not exist") + } + } + + /** + * This is the function used to create OAP index. It may be called several time to build + * one OAP index file. We need to store value, rowId, indexFileName for each data record. + * + * Since we use ES Index to store all indexes for one table, we need an identifier to locate + * one specific OAP index file. So, data file name and column name is needed. + * + * @param records From data file, it's an array of (value, rowId) pair + * @param oapIndexFileName Index File name is the composition of indexName and dataFileName + * @param tableName table name (corresponding to one ES table index) + * @return successfully inserted row count + */ + override def batchInsert( + records: Seq[(Key, Int)], oapIndexFileName: String, tableName: String): Int = { + + val bulkRequest = client.prepareBulk() + + records.foreach { case (key, rowId) => + val obj = XContentFactory.jsonBuilder() + .startObject() + .field("key", key.getUTF8String(0)) + .field("index_file_name", oapIndexFileName) + .field("rowId", rowId) + .endObject() + val indexRequest = client.prepareIndex(tableName, docType) + .setSource(obj) + bulkRequest.add(indexRequest) + } + bulkRequest.execute().actionGet() + client.admin().indices().refresh(new RefreshRequest(tableName)).actionGet() + records.length + } + + def queryAllRecordIds(tableIndexName: String): Seq[String] = { + val queryBuilder = QueryBuilders.matchAllQuery() + val response = client.prepareSearch(tableIndexName) + .setTypes(docType) + .addDocValueField("rowId") + .addDocValueField("index_file_name") + .setQuery(queryBuilder).setSize(100).execute().actionGet() + + response.getHits.getHits.map(_.getId) + } + + def queryIdsByIndexFileName(oapIndexFileName: String, tableIndexName: String): Seq[String] = { + + val queryBuilder = QueryBuilders.termQuery("index_file_name", oapIndexFileName) + val response = client.prepareSearch(tableIndexName) + .setTypes(docType) + .addDocValueField("rowId") + .setQuery(queryBuilder).setSize(100).execute().actionGet() + + response.getHits.getHits.map(_.getField("rowId").getValue.toString) + } + + /** + * Pick one idle connection from connection pool + * TODO: Should be a pool + * + * @return the idle ES TransportClient + */ + override protected def getEsClient: TransportClient = { + val settings = Settings.builder + .put("cluster.name", "elasticsearch") + .put("client.transport.sniff", true) + .put("client.transport.ignore_cluster_name", false) + .put("client.transport.ping_timeout", "5s") + .put("client.transport.nodes_sampler_interval", "5s").build + + val client = new OapTransportClient(settings) + val address = new InetSocketTransportAddress(InetAddress.getByName(host), port) + + client.addTransportAddress(address) + client.connectedNodes() + + client + } + + /** + * create an ES index for one OAP table + * ES Table Index is not equal to OAP index + * All OAP indexes built for one table is stored in one ES Table Index file + * We only create ES Table Index once for each table + * + * @param tableName table name (corresponding to one ES table index) + */ + override def createEsTableIndex(tableName: String): Unit = { + + if (!checkEsTableIndexExists(tableName)) { + + client.admin().indices() + .prepareCreate(tableName) + .setSource(mappings) + .execute().actionGet() + } else { + logWarning(s"ES Index: $tableName is already exist") + } + } + + /** + * Check if the ES table index has already been created + * + * @param tableName table name (corresponding to one ES table index) + * @return true if the index is already created. + */ + override def checkEsTableIndexExists(tableName: String): Boolean = { + client.admin().indices() + .exists(new IndicesExistsRequest(tableName)) + .actionGet().isExists + } + + /** + * Drop all ES Index records belong to one OAP index file (dataFileName + indexName) + * + * @param oapIndexFileName Index File name is the composition of indexName and dataFileName + * @param tableName table name (corresponding to one ES table index) + * @return successfully deleted row count + */ + override def batchDelete(oapIndexFileName: String, tableName: String): Int = { + val ids = queryIdsByIndexFileName(oapIndexFileName, tableName) + + val bulkRequest = client.prepareBulk() + + ids.foreach { id => + bulkRequest.add(client.prepareDelete(tableName, docType, id.toString)) + } + bulkRequest.execute().actionGet() + client.admin().indices().refresh(new RefreshRequest(tableName)).actionGet() + ids.length + } + + /** + * Drop all ES Index records belong to some OAP index files (dataFileName + indexName) + * + * @param oapIndexFileNames Array of Index File names + * @param tableName table name (corresponding to one ES table index) + * @return successfully deleted row count + */ + override def batchDelete(oapIndexFileNames: Seq[String], tableName: String): Int = { + oapIndexFileNames.map(batchDelete(_, tableName)).sum + } + + /** + * Use ES scroll fetch API to return results. It may need several times to fetch all results + * + * @param query query + * @param oapIndexFileName Index File name is the composition of indexName and dataFileName + * @return row ids satisfy the query + */ + override def scrollFetch(query: String, oapIndexFileName: String): Seq[Int] = + throw new NotImplementedError() + + /** + * Fetch all results + * + * @param query query + * @param oapIndexFileName Index File name is the composition of indexName and dataFileName + * @return row ids satisfy the query + */ + override def fullFetch(query: String, oapIndexFileName: String): Seq[Int] = + throw new NotImplementedError() +} +*/ diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapEsUtil.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapEsUtil.scala new file mode 100644 index 00000000000..33383a26ed4 --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapEsUtil.scala @@ -0,0 +1,97 @@ +/* + * 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.datasources.oap.index + +import org.elasticsearch.client.transport.TransportClient + +import org.apache.spark.sql.execution.datasources.oap.Key + +/** + * This should be a wrapper of [[TransportClient]], providing several APIs to access ES server. + */ +trait OapEsUtil { + + /** + * client to send requests to ES server + */ + protected def transportClient: TransportClient + + /** + * create an ES index for one OAP table + * ES Table Index is not equal to OAP index + * All OAP indexes built for one table is stored in one ES Table Index file + * We only create ES Table Index once for each table + * + * @param tableName table name (corresponding to one ES table index) + */ + def createEsTableIndex(tableName: String): Unit + + /** + * Check if the ES table index has already been created + * + * @param tableName table name (corresponding to one ES table index) + * @return true if the index is already created. + */ + protected def checkEsTableIndexExists(tableName: String): Boolean + + /** + * This is the function used to create OAP index. It may be called several time to build + * one OAP index file. We need to store value, rowId, indexFileName for each data record. + * + * @param records From data file, it's an array of (value, rowId) pair + * @param oapIndexFileName Index File name is the composition of indexName and dataFileName + * @param tableName table name (corresponding to one ES table index) + * @return successfully inserted row count + */ + def batchInsert(records: Seq[(Key, Int)], oapIndexFileName: String, tableName: String): Int + + /** + * Drop all ES Index records belong to one OAP index file (dataFileName + indexName) + * + * @param oapIndexFileName Index File name is the composition of indexName and dataFileName + * @param tableName table name (corresponding to one ES table index) + * @return successfully deleted row count + */ + def batchDelete(oapIndexFileName: String, tableName: String): Int + + /** + * Drop all ES Index records belong to some OAP index files (dataFileName + indexName) + * @param oapIndexFileNames Array of Index File names + * @param tableName table name (corresponding to one ES table index) + * @return successfully deleted row count + */ + def batchDelete(oapIndexFileNames: Seq[String], tableName: String): Int + + /** + * Use ES scroll fetch API to return results. It may need several times to fetch all results + * + * @param query query + * @param oapIndexFileName Index File name is the composition of indexName and dataFileName + * @return row ids satisfy the query + */ + def scrollFetch(query: String, oapIndexFileName: String): Seq[Int] + + /** + * Fetch all results + * + * @param query query + * @param oapIndexFileName Index File name is the composition of indexName and dataFileName + * @return row ids satisfy the query + */ + def fullFetch(query: String, oapIndexFileName: String): Seq[Int] +} diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexOutputFormat.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexOutputFormat.scala index bb2d8d7bc89..05ba852f768 100644 --- a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexOutputFormat.scala +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexOutputFormat.scala @@ -22,10 +22,10 @@ import org.apache.hadoop.mapreduce.{RecordWriter, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.parquet.format.CompressionCodec import org.apache.parquet.hadoop.util.ContextUtil - import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.OapException import org.apache.spark.sql.execution.datasources.oap.index.OapIndexProperties.IndexVersion +import org.apache.spark.sql.execution.datasources.oap.index.elasticsearch.EsIndexRecordWriter import org.apache.spark.sql.internal.oap.OapConf import org.apache.spark.sql.types.StructType @@ -83,6 +83,8 @@ private[index] class OapIndexOutputFormat extends FileOutputFormat[Void, Interna } else if (indexType == "BITMAP") { val writer = file.getFileSystem(configuration).create(file, true) new BitmapIndexRecordWriter(configuration, writer, schema) + } else if (indexType == "ES") { + new EsIndexRecordWriter(configuration, file, schema) } else { throw new OapException("Unknown Index Type: " + indexType) } diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexType.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexType.scala index 252fcce7bf5..123834b49c6 100644 --- a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexType.scala +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexType.scala @@ -28,3 +28,7 @@ case object BTreeIndexType extends OapIndexType { case object BitMapIndexType extends OapIndexType { override def toString: String = "BITMAP" } + +case object ESIndexType extends OapIndexType { + override def toString: String = "ES" +} diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/UnsafeIndexTree.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/UnsafeIndexTree.scala index 4da042ee2a2..052102a7695 100644 --- a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/UnsafeIndexTree.scala +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/UnsafeIndexTree.scala @@ -225,7 +225,7 @@ private[oap] class RangeInterval( var startInclude = includeStart var endInclude = includeEnd val isNullPredicate = isNull - val isPrefixMatch = ignoreTail + var isPrefixMatch = ignoreTail } private[oap] object RangeInterval{ diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/ESScanner.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/ESScanner.scala new file mode 100644 index 00000000000..d8791ba06af --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/ESScanner.scala @@ -0,0 +1,51 @@ +/* + * 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.datasources.oap.index.elasticsearch + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.execution.datasources.oap.IndexMeta +import org.apache.spark.sql.execution.datasources.oap.index.{IndexScanner, IndexUtils} + +case class ESScanner(idxMeta: IndexMeta) extends IndexScanner(idxMeta) { + + // TODO: return all rows in one data file + override def totalRows(): Long = _totalRows + + @transient private var internalIter: Iterator[Int] = _ + @transient private var _totalRows: Long = _ + + override def initialize(dataPath: Path, conf: Configuration): IndexScanner = { + + val client = OapEsClientRepository.getOrCreateOapEsClient() + + val indexPath = IndexUtils.getIndexFilePath(conf, dataPath, meta.name, meta.time) + + val rowIds = client.fullFetch("", indexPath.getName, OapEsProperties.ES_INDEX_NAME) + _totalRows = rowIds.length + + internalIter = rowIds.toIterator + + this + } + + override def hasNext: Boolean = internalIter.hasNext + + override def next(): Int = internalIter.next() +} diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/EsIndexRecordWriter.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/EsIndexRecordWriter.scala new file mode 100644 index 00000000000..06cd781b639 --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/EsIndexRecordWriter.scala @@ -0,0 +1,58 @@ +/* + * 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.datasources.oap.index.elasticsearch + +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.{RecordWriter, TaskAttemptContext} + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.types.StructType + +class EsIndexRecordWriter( + configuration: Configuration, + indexPath: Path, + keySchema: StructType) extends RecordWriter[Void, InternalRow] { + + private val client = OapEsClientRepository.getOrCreateOapEsClient() + private var recordCount = 0 + + private val bufferedRecords = new ArrayBuffer[(String, Int)]() + + private def flushBufferedRows(): Unit = { + client.batchInsert(bufferedRecords, indexPath.getName, OapEsProperties.ES_INDEX_NAME) + bufferedRecords.clear() + } + + override def close(taskAttemptContext: TaskAttemptContext): Unit = { + flushBufferedRows() + // Just touch a file without any content + val file = indexPath.getFileSystem(configuration).create(indexPath) + file.close() + } + + override def write(k: Void, v: InternalRow): Unit = { + if (bufferedRecords.length == 100) { + flushBufferedRows() + } + bufferedRecords.append((v.get(0, keySchema.head.dataType).toString, recordCount)) + recordCount += 1 + } +} diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsClient.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsClient.scala new file mode 100644 index 00000000000..a688cb6db38 --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsClient.scala @@ -0,0 +1,103 @@ +/* + * 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.datasources.oap.index.elasticsearch + +import org.elasticsearch.{ResourceAlreadyExistsException, ResourceNotFoundException} +import org.elasticsearch.client.transport.TransportClient + +trait OapEsClient { + + /** + * client to send requests to ES server + */ + protected def transportClient: TransportClient + + /** + * create an ES index. + * + * In OAP, each data file is related to N index files, which N is the index number. + * But in ES, it's different. We only use 1 index in ES to store all records from OAP. + * Each record will have 3 fields in ES: row_id, value, oap_index_file_name. + * `oap_index_file_name` is exactly same to OAP's BTREE and BITMAP index file name. + * We use this to indicate the record is from which data file and which column + * + * @param indexName index which stores OAP records + * @throws ResourceAlreadyExistsException if indexName is already exist + */ + protected[elasticsearch] def createEsIndex(indexName: String): Unit + + /** + * Drop ES index. This will delete all data inserted by OAP. + * + * @param indexName index which stores OAP records* + * @throws ResourceNotFoundException if indexName is not exist + */ + protected[elasticsearch] def dropEsIndex(indexName: String): Unit + + /** + * Check if the ES table index has already been created + * + * @param indexName index which stores OAP records + * @return true if the index is already created. + */ + protected[elasticsearch] def checkEsIndexExists(indexName: String): Boolean + + /** + * Insert records into ES. Inserted records will be indexed by ES. + * When OAP creates index, it will generate an index file name using a combination with + * data file name and index name. Use this info, we will know the records from ES is from + * which data file and which column. + * For all record values, should be string type. + * + * @param records From data file, it's an array of (value, rowId) pair + * @param oapIndexFileName Index File name is the composition of indexName and dataFileName + * @param indexName index which stores OAP records + * @return successfully inserted row count + */ + def batchInsert(records: Seq[(String, Int)], oapIndexFileName: String, indexName: String): Int + + /** + * Drop all ES Index records belong to one OAP index file (dataFileName + indexName) + * + * @param oapIndexFileName Index File name is the composition of indexName and dataFileName + * @param indexName index which stores OAP records + * @return successfully deleted row count + */ + def batchDelete(oapIndexFileName: String, indexName: String): Int + + /** + * Use ES scroll fetch API to return results. It may need several times to fetch all results + * + * @param query query + * @param oapIndexFileName Index File name is the composition of indexName and dataFileName + * @param indexName index which stores OAP records + * @return row ids satisfy the query + */ + def scrollFetch(query: String, oapIndexFileName: String, indexName: String): Seq[Int] + + /** + * Fetch all results + * + * @param query query + * @param oapIndexFileName Index File name is the composition of indexName and dataFileName + * @param indexName index which stores OAP records + * @return row ids satisfy the query + */ + def fullFetch(query: String, oapIndexFileName: String, indexName: String): Seq[Int] +} + diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsClientImpl.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsClientImpl.scala new file mode 100644 index 00000000000..0cb7793ad02 --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsClientImpl.scala @@ -0,0 +1,104 @@ +/* + * 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.datasources.oap.index.elasticsearch + +import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest +import org.elasticsearch.action.admin.indices.exists.indices.IndicesExistsRequest +import org.elasticsearch.action.admin.indices.refresh.RefreshRequest +import org.elasticsearch.client.transport.TransportClient +import org.elasticsearch.common.xcontent.{XContentBuilder, XContentFactory} +import org.elasticsearch.index.query.QueryBuilders + +case class OapEsClientImpl(transportClient: TransportClient) extends OapEsClient { + + // TODO: Change mappings to Map + override protected[elasticsearch] + def createEsIndex(indexName: String): Unit = { + + transportClient.admin().indices() + .prepareCreate(indexName) + .setSource(OapEsProperties.mappings) + .execute().actionGet() + } + + override protected[elasticsearch] + def dropEsIndex(indexName: String): Unit = { + transportClient.admin().indices() + .delete(new DeleteIndexRequest(indexName)) + .actionGet() + } + + override protected[elasticsearch] + def checkEsIndexExists(indexName: String): Boolean = { + transportClient.admin().indices() + .exists(new IndicesExistsRequest(indexName)) + .actionGet().isExists + } + + override def batchInsert( + records: Seq[(String, Int)], oapIndexFileName: String, indexName: String): Int = { + + val bulkRequest = transportClient.prepareBulk() + + records.foreach { case (value, rowId) => + val obj = XContentFactory.jsonBuilder() + .startObject() + .field(OapEsProperties.VALUE_FIELD, value) + .field(OapEsProperties.INDEX_FILE_NAME_FIELD, oapIndexFileName) + .field(OapEsProperties.ROW_ID_FIELD, rowId) + .endObject() + val indexRequest = transportClient.prepareIndex(indexName, OapEsProperties.DOC_TYPE) + .setSource(obj) + bulkRequest.add(indexRequest) + } + bulkRequest.execute().actionGet() + transportClient.admin().indices().refresh(new RefreshRequest(indexName)).actionGet() + records.length + } + + private def queryIdsByIndexFileName(oapIndexFileName: String, indexName: String): Seq[String] = { + val queryBuilder = QueryBuilders.termQuery( + OapEsProperties.INDEX_FILE_NAME_FIELD, oapIndexFileName) + + val response = transportClient.prepareSearch(indexName) + .setTypes(OapEsProperties.DOC_TYPE) + .setQuery(queryBuilder).setSize(100).execute().actionGet() + + response.getHits.getHits.map(_.getId) + } + + override def batchDelete(oapIndexFileName: String, indexName: String): Int = { + val ids = queryIdsByIndexFileName(oapIndexFileName, indexName) + + val bulkRequest = transportClient.prepareBulk() + + ids.foreach { id => + bulkRequest.add(transportClient.prepareDelete(indexName, OapEsProperties.DOC_TYPE, id)) + } + bulkRequest.execute().actionGet() + transportClient.admin().indices().refresh(new RefreshRequest(indexName)).actionGet() + ids.length + } + + override def scrollFetch( + query: String, oapIndexFileName: String, indexName: String): Seq[Int] = Seq.empty + + override def fullFetch( + query: String, oapIndexFileName: String, indexName: String): Seq[Int] = Seq.empty + +} diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsClientRepository.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsClientRepository.scala new file mode 100644 index 00000000000..54bd374a360 --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsClientRepository.scala @@ -0,0 +1,70 @@ +/* + * 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.datasources.oap.index.elasticsearch + +import java.net.InetAddress + +import org.elasticsearch.client.transport.TransportClient +import org.elasticsearch.common.settings.Settings +import org.elasticsearch.common.transport.InetSocketTransportAddress + +import org.apache.spark.SparkEnv +import org.apache.spark.sql.execution.datasources.parquet.OapTransportClient +import org.apache.spark.sql.internal.oap.OapConf + +object OapEsClientRepository { + + private val esHost = SparkEnv.get.conf.get(OapConf.OAP_ELASTICSEARCH_HOST) + private val esPort = SparkEnv.get.conf.get(OapConf.OAP_ELASTICSEARCH_PORT) + + private val esClientSettings = { + Settings.builder + .put("cluster.name", "elasticsearch") + .put("client.transport.sniff", true) + .put("client.transport.ignore_cluster_name", false) + .put("client.transport.ping_timeout", "5s") + .put("client.transport.nodes_sampler_interval", "5s").build + } + +// initialization() +// +// private def initialization() { +// // Before starting, let's check if we have created an index for oap +// val client = getOrCreateOapEsClient() +// if (client.checkEsIndexExists(OapEsProperties.ES_INDEX_NAME)) { +// client.createEsIndex(OapEsProperties.ES_INDEX_NAME) +// } +// } + + // Currently just return a new client instance. + // TODO: we need a client pool for recycling. + def getOrCreateOapEsClient(): OapEsClient = { + + val transportClient = new OapTransportClient(esClientSettings) + val address = new InetSocketTransportAddress(InetAddress.getByName(esHost), esPort) + + transportClient.addTransportAddress(address) + transportClient.connectedNodes() + + val client = OapEsClientImpl(transportClient) + if (!client.checkEsIndexExists(OapEsProperties.ES_INDEX_NAME)) { + client.createEsIndex(OapEsProperties.ES_INDEX_NAME) + } + client + } +} diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsProperties.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsProperties.scala new file mode 100644 index 00000000000..3dcff39692c --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsProperties.scala @@ -0,0 +1,62 @@ +/* + * 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.datasources.oap.index.elasticsearch + +import org.elasticsearch.common.xcontent.XContentFactory + +object OapEsProperties { + val ES_INDEX_NAME = "all_oap_es_index" + val DOC_TYPE = "oap_records" + val VALUE_FIELD = "value" + val ROW_ID_FIELD = "row_id" + val INDEX_FILE_NAME_FIELD = "index_file_name" + + // TODO: Change this to a Map[String, Any] + val mappings = XContentFactory.jsonBuilder() + .startObject() + // Settings of this Index + .startObject("settings") + .field("number_of_shards", 1) + .field("number_of_replicas", 0) + .endObject() + .startObject("mappings") + // This is the document type, called "oap_records" + .startObject(DOC_TYPE) + .startObject("properties") + // value in indexed column. + .startObject(VALUE_FIELD) + .field("type", "text") + .field("store", "yes") + .field("index", "analyzed") + .endObject() + // index file name = data file name + index name + .startObject(INDEX_FILE_NAME_FIELD) + .field("type", "string") + .field("store", "yes") + .field("index", "not_analyzed") + .endObject() + // The row id of this record in data file + .startObject(ROW_ID_FIELD) + .field("type", "integer") + .field("store", "yes") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() +} diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/indexPlans.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/indexPlans.scala index f5844a476c2..efc09d807bd 100644 --- a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/indexPlans.scala +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/indexPlans.scala @@ -134,6 +134,13 @@ case class CreateIndexCommand( val entries = indexColumns.map(col => schema.map(_.name).toIndexedSeq.indexOf(col.columnName)) metaBuilder.addIndexMeta(new IndexMeta(indexName, time, BitMapIndex(entries))) + case ESIndexType => + if (indexColumns.length != 1) { + throw new OapException("ESIndexType only supports one single column") + } + val entries = indexColumns.map(col => + schema.map(_.name).toIndexedSeq.indexOf(col.columnName)) + metaBuilder.addIndexMeta(new IndexMeta(indexName, time, ESIndex(entries))) case _ => sys.error(s"Not supported index type $indexType") } @@ -188,7 +195,7 @@ case class CreateIndexCommand( outputSpec = FileFormatWriter.OutputSpec( qualifiedOutputPath.toUri.getPath, Map.empty), hadoopConf = configuration, - partitionColumns = Seq.empty, + Seq.empty, // partitionColumns bucketSpec = Option.empty, refreshFunction = _ => Unit, options = options).asInstanceOf[Seq[Seq[IndexBuildResult]]] @@ -426,7 +433,7 @@ case class RefreshIndexCommand( outputSpec = FileFormatWriter.OutputSpec( qualifiedOutputPath.toUri.getPath, Map.empty), hadoopConf = configuration, - partitionColumns = Seq.empty, + Seq.empty, // partitionColumns bucketSpec = Option.empty, refreshFunction = _ => Unit, options = options).asInstanceOf[Seq[Seq[IndexBuildResult]]] diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/CaseInsensitiveMap.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/CaseInsensitiveMap.scala new file mode 100644 index 00000000000..f175fc42742 --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/CaseInsensitiveMap.scala @@ -0,0 +1,54 @@ +/* + * 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.datasources.oap.utils + +import java.util.Locale + +/** + * Builds a map in which keys are case insensitive. Input map can be accessed for cases where + * case-sensitive information is required. The primary constructor is marked private to avoid + * nested case-insensitive map creation, otherwise the keys in the original map will become + * case-insensitive in this scenario. + */ +class CaseInsensitiveMap[T] private (val originalMap: Map[String, T]) extends Map[String, T] + with Serializable { + + val keyLowerCasedMap = originalMap.map(kv => kv.copy(_1 = kv._1.toLowerCase(Locale.ROOT))) + + override def get(k: String): Option[T] = keyLowerCasedMap.get(k.toLowerCase(Locale.ROOT)) + + override def contains(k: String): Boolean = + keyLowerCasedMap.contains(k.toLowerCase(Locale.ROOT)) + + override def +[B1 >: T](kv: (String, B1)): Map[String, B1] = { + new CaseInsensitiveMap(originalMap + kv) + } + + override def iterator: Iterator[(String, T)] = keyLowerCasedMap.iterator + + override def -(key: String): Map[String, T] = { + new CaseInsensitiveMap(originalMap.filterKeys(!_.equalsIgnoreCase(key))) + } +} + +object CaseInsensitiveMap { + def apply[T](params: Map[String, T]): CaseInsensitiveMap[T] = params match { + case caseSensitiveMap: CaseInsensitiveMap[T] => caseSensitiveMap + case _ => new CaseInsensitiveMap(params) + } +} diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/FilterHelper.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/FilterHelper.scala index db91f934083..0b52c86166b 100644 --- a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/FilterHelper.scala +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/FilterHelper.scala @@ -22,7 +22,7 @@ import org.apache.parquet.filter2.predicate.{FilterApi, FilterPredicate} import org.apache.parquet.hadoop.ParquetInputFormat import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.datasources.parquet.ParquetFilters +import org.apache.spark.sql.execution.datasources.parquet.ParquetFiltersWrapper import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType @@ -44,7 +44,7 @@ object FilterHelper { // Collects all converted Parquet filter predicates. Notice that not all predicates can be // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` // is used here. - .flatMap(ParquetFilters.createFilter(requiredSchema, _)) + .flatMap(ParquetFiltersWrapper.createFilter(requiredSchema, _)) .reduceOption(FilterApi.and) } else { None diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/OapIndexInfoStatusSerDe.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/OapIndexInfoStatusSerDe.scala index b5175700c69..c66273a5269 100644 --- a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/OapIndexInfoStatusSerDe.scala +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/OapIndexInfoStatusSerDe.scala @@ -34,6 +34,8 @@ import org.apache.spark.sql.execution.datasources.oap.io.OapIndexInfoStatus private[oap] object OapIndexInfoStatusSerDe extends SerDe[String, Seq[OapIndexInfoStatus]] { import org.json4s.jackson.JsonMethods._ + private implicit val format = DefaultFormats + override def serialize(indexStatusRawDataArray: Seq[OapIndexInfoStatus]): String = { val statusJArray = JArray(indexStatusRawDataArray.map(indexStatusRawDataToJson).toList) compact(render("oapIndexInfoStatusRawDataArray" -> statusJArray)) @@ -41,11 +43,9 @@ private[oap] object OapIndexInfoStatusSerDe extends SerDe[String, Seq[OapIndexIn override def deserialize(json: String): Seq[OapIndexInfoStatus] = { (parse(json) \ "oapIndexInfoStatusRawDataArray").extract[List[JValue]].map( - indexStatusRawDataFromJson) + indexStatusRawDataFromJson(_)) } - private implicit val format = DefaultFormats - private[oap] def indexStatusRawDataFromJson(json: JValue): OapIndexInfoStatus = { val path = (json \ "partitionFilePath").extract[String] val useIndex = (json \ "useOapIndex").extract[Boolean] diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/OapUtils.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/OapUtils.scala index 7d5bbdf529d..1322f7a61f6 100644 --- a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/OapUtils.scala +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/oap/utils/OapUtils.scala @@ -27,13 +27,13 @@ import org.apache.parquet.bytes.BytesUtils import org.apache.parquet.io.api.Binary import org.apache.spark.internal.Logging -import org.apache.spark.sql.{AnalysisException, RuntimeConfig} +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.catalog.CatalogTypes._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} import org.apache.spark.sql.execution.datasources.{FileIndex, PartitionDirectory, PartitioningUtils} import org.apache.spark.sql.execution.datasources.oap.{DataSourceMeta, Key, OapFileFormat} -import org.apache.spark.sql.internal.oap.OapConf +import org.apache.spark.sql.oap.adapter.FileIndexAdapter import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -87,7 +87,7 @@ object OapUtils extends Logging { } else { Nil } - fileIndex.listFiles(filters) + FileIndexAdapter.listFiles(fileIndex, filters, Nil) } /** diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFiltersWrapper.scala b/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFiltersWrapper.scala new file mode 100644 index 00000000000..45c0b60ff34 --- /dev/null +++ b/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFiltersWrapper.scala @@ -0,0 +1,32 @@ +/* + * 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.datasources.parquet + +import org.apache.parquet.filter2.predicate.FilterPredicate + +import org.apache.spark.sql.sources +import org.apache.spark.sql.types.StructType + +/** + * Wrap ParquetFilters so it can be accessed outside of the parquet package. + */ +object ParquetFiltersWrapper { + def createFilter(schema: StructType, predicate: sources.Filter): Option[FilterPredicate] = { + ParquetFilters.createFilter(schema, predicate) + } +} diff --git a/src/main/scala/org/apache/spark/sql/internal/oap/OapConf.scala b/src/main/scala/org/apache/spark/sql/internal/oap/OapConf.scala index 65f23576560..7f6a967a8fe 100644 --- a/src/main/scala/org/apache/spark/sql/internal/oap/OapConf.scala +++ b/src/main/scala/org/apache/spark/sql/internal/oap/OapConf.scala @@ -17,23 +17,25 @@ package org.apache.spark.sql.internal.oap +import org.apache.spark.internal.config.ConfigBuilder +import org.apache.spark.sql.oap.adapter.SqlConfAdapter + //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines the configuration options for OAP. //////////////////////////////////////////////////////////////////////////////////////////////////// object OapConf { - import org.apache.spark.sql.internal.SQLConf.SQLConfigBuilder val OAP_PARQUET_ENABLED = - SQLConfigBuilder("spark.sql.oap.parquet.enable") + SqlConfAdapter.buildConf("spark.sql.oap.parquet.enable") .internal() .doc("Whether enable oap file format when encounter parquet files") .booleanConf .createWithDefault(true) val OAP_FULL_SCAN_THRESHOLD = - SQLConfigBuilder("spark.sql.oap.statistics.fullScanThreshold") + SqlConfAdapter.buildConf("spark.sql.oap.statistics.fullScanThreshold") .internal() .doc("Define the full scan threshold based on oap statistics in index file. " + "If the analysis result is above this threshold, it will full scan data file, " + @@ -42,7 +44,7 @@ object OapConf { .createWithDefault(0.2) val OAP_STATISTICS_TYPES = - SQLConfigBuilder("spark.sql.oap.statistics.type") + SqlConfAdapter.buildConf("spark.sql.oap.statistics.type") .internal() .doc("Which types of pre-defined statistics are added in index file. " + "And here you should just write the statistics name. " + @@ -61,28 +63,28 @@ object OapConf { .createWithDefault(Seq("BLOOM", "MINMAX", "PARTBYVALUE", "SAMPLE")) val OAP_STATISTICS_PART_NUM = - SQLConfigBuilder("spark.sql.oap.statistics.partNum") + SqlConfAdapter.buildConf("spark.sql.oap.statistics.partNum") .internal() .doc("PartedByValueStatistics gives statistics with the value interval, default 5") .intConf .createWithDefault(5) val OAP_STATISTICS_SAMPLE_RATE = - SQLConfigBuilder("spark.sql.oap.statistics.sampleRate") + SqlConfAdapter.buildConf("spark.sql.oap.statistics.sampleRate") .internal() .doc("Sample rate for sample based statistics, default value 0.05") .doubleConf .createWithDefault(0.05) val OAP_STATISTICS_SAMPLE_MIN_SIZE = - SQLConfigBuilder("spark.sql.oap.statistics.sampleMinSize") + SqlConfAdapter.buildConf("spark.sql.oap.statistics.sampleMinSize") .internal() .doc("Minimum sample size for Sample Statistics, default value 24") .intConf .createWithDefault(24) val OAP_BLOOMFILTER_MAXBITS = - SQLConfigBuilder("spark.sql.oap.statistics.bloom.maxBits") + SqlConfAdapter.buildConf("spark.sql.oap.statistics.bloom.maxBits") .internal() .doc("Define the max bit count parameter used in bloom " + "filter, default 33554432") @@ -90,34 +92,34 @@ object OapConf { .createWithDefault(1 << 20) val OAP_BLOOMFILTER_NUMHASHFUNC = - SQLConfigBuilder("spark.sql.oap.statistics.bloom.numHashFunc") + SqlConfAdapter.buildConf("spark.sql.oap.statistics.bloom.numHashFunc") .internal() .doc("Define the number of hash functions used in bloom filter, default 3") .intConf .createWithDefault(3) val OAP_FIBERCACHE_SIZE = - SQLConfigBuilder("spark.sql.oap.fiberCache.size") + SqlConfAdapter.buildConf("spark.sql.oap.fiberCache.size") .internal() .doc("Define the size of fiber cache in KB, default 300 * 1024 KB") .longConf .createWithDefault(307200) val OAP_FIBERCACHE_STATS = - SQLConfigBuilder("spark.sql.oap.fiberCache.stats") + SqlConfAdapter.buildConf("spark.sql.oap.fiberCache.stats") .internal() .doc("Whether enable cach stats record, default false") .booleanConf .createWithDefault(false) val OAP_FIBERCACHE_USE_OFFHEAP_RATIO = - SQLConfigBuilder("spark.sql.oap.fiberCache.use.offheap.ratio") + SqlConfAdapter.buildConf("spark.sql.oap.fiberCache.use.offheap.ratio") .internal() .doc("Define the ratio of fiber cache use 'spark.memory.offHeap.size' ratio.") .doubleConf .createWithDefault(0.7) - val OAP_COMPRESSION = SQLConfigBuilder("spark.sql.oap.compression.codec") + val OAP_COMPRESSION = SqlConfAdapter.buildConf("spark.sql.oap.compression.codec") .internal() .doc("Sets the compression codec use when writing Parquet files. Acceptable values include: " + "uncompressed, snappy, gzip, lzo.") @@ -126,7 +128,8 @@ object OapConf { .checkValues(Set("UNCOMPRESSED", "SNAPPY", "GZIP", "LZO")) .createWithDefault("GZIP") - val OAP_INDEX_BTREE_COMPRESSION = SQLConfigBuilder("spark.sql.oap.index.compression.codec") + val OAP_INDEX_BTREE_COMPRESSION = + SqlConfAdapter.buildConf("spark.sql.oap.index.compression.codec") .internal() .doc("Sets the compression codec use when writing Parquet files. Acceptable values include: " + "uncompressed, snappy, gzip, lzo.") @@ -136,70 +139,70 @@ object OapConf { .createWithDefault("GZIP") val OAP_ROW_GROUP_SIZE = - SQLConfigBuilder("spark.sql.oap.rowgroup.size") + SqlConfAdapter.buildConf("spark.sql.oap.rowgroup.size") .internal() .doc("Define the row number for each row group") .intConf .createWithDefault(1024 * 1024) val OAP_ENABLE_OINDEX = - SQLConfigBuilder("spark.sql.oap.oindex.enabled") + SqlConfAdapter.buildConf("spark.sql.oap.oindex.enabled") .internal() .doc("To indicate to enable/disable oindex for developers even if the index file is there") .booleanConf .createWithDefault(true) val OAP_ENABLE_EXECUTOR_INDEX_SELECTION = - SQLConfigBuilder("spark.sql.oap.oindex.eis.enabled") + SqlConfAdapter.buildConf("spark.sql.oap.oindex.eis.enabled") .internal() .doc("To indicate if enable/disable index cbo which helps to choose a fast query path") .booleanConf .createWithDefault(true) val OAP_EXECUTOR_INDEX_SELECTION_FILE_POLICY = - SQLConfigBuilder("spark.sql.oap.oindex.file.policy") + SqlConfAdapter.buildConf("spark.sql.oap.oindex.file.policy") .internal() .doc("To indicate if enable/disable file based index selection") .booleanConf .createWithDefault(true) val OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY = - SQLConfigBuilder("spark.sql.oap.oindex.statistics.policy") + SqlConfAdapter.buildConf("spark.sql.oap.oindex.statistics.policy") .internal() .doc("To indicate if enable/disable statistics based index selection") .booleanConf .createWithDefault(true) val OAP_ENABLE_OPTIMIZATION_STRATEGIES = - SQLConfigBuilder("spark.sql.oap.strategies.enabled") + SqlConfAdapter.buildConf("spark.sql.oap.strategies.enabled") .internal() .doc("To indicate if enable/disable oap strategies") .booleanConf .createWithDefault(false) val OAP_INDEX_FILE_SIZE_MAX_RATIO = - SQLConfigBuilder("spark.sql.oap.oindex.size.ratio") + SqlConfAdapter.buildConf("spark.sql.oap.oindex.size.ratio") .internal() .doc("To indicate if enable/disable index cbo which helps to choose a fast query path") .doubleConf .createWithDefault(0.7) val OAP_INDEXER_CHOICE_MAX_SIZE = - SQLConfigBuilder("spark.sql.oap.indexer.max.use.size") + SqlConfAdapter.buildConf("spark.sql.oap.indexer.max.use.size") .internal() .doc("The max availabe indexer choose size.") .intConf .createWithDefault(1) val OAP_BTREE_ROW_LIST_PART_SIZE = - SQLConfigBuilder("spark.sql.oap.btree.rowList.part.size") + SqlConfAdapter.buildConf("spark.sql.oap.btree.rowList.part.size") .internal() .doc("The row count of each part of row list in btree index") .intConf .createWithDefault(1024 * 1024) val OAP_INDEX_DISABLE_LIST = - SQLConfigBuilder("spark.sql.oap.oindex.disable.list") + SqlConfAdapter.buildConf("spark.sql.oap.oindex.disable.list") .internal() .doc("To disable specific index by index names for test purpose, this is supposed to be in " + "the format of indexA,indexB,indexC") @@ -207,21 +210,21 @@ object OapConf { .createWithDefault("") val OAP_HEARTBEAT_INTERVAL = - SQLConfigBuilder("spark.sql.oap.heartbeatInterval") + SqlConfAdapter.buildConf("spark.sql.oap.heartbeatInterval") .internal() .doc("To Configure the OAP status update interval, for example OAP metrics") .stringConf .createWithDefault("2s") val OAP_UPDATE_FIBER_CACHE_METRICS_INTERVAL_SEC = - SQLConfigBuilder("spark.sql.oap.update.fiber.cache.metrics.interval.sec") + SqlConfAdapter.buildConf("spark.sql.oap.update.fiber.cache.metrics.interval.sec") .internal() .doc("The interval of fiber cache metrics update") .longConf .createWithDefault(10L) val OAP_INDEX_BTREE_WRITER_VERSION = - SQLConfigBuilder("spark.sql.oap.index.btree.writer.version") + SqlConfAdapter.buildConf("spark.sql.oap.index.btree.writer.version") .internal() .doc("The writer version of BTree index") .stringConf @@ -229,17 +232,31 @@ object OapConf { .createWithDefault("v1") val OAP_PARQUET_DATA_CACHE_ENABLED = - SQLConfigBuilder("spark.sql.oap.parquet.data.cache.enable") + SqlConfAdapter.buildConf("spark.sql.oap.parquet.data.cache.enable") .internal() .doc("To indicate if enable parquet data cache, default false") .booleanConf .createWithDefault(false) val OAP_INDEX_DIRECTORY = - SQLConfigBuilder("spark.sql.oap.index.directory") + SqlConfAdapter.buildConf("spark.sql.oap.index.directory") .internal() .doc("To specify the directory of index file, if the value " + "is empty, it will store in the data file path") .stringConf .createWithDefault("") + + val OAP_ELASTICSEARCH_HOST = + SqlConfAdapter.buildConf("spark.sql.oap.elasticsearch.host") + .internal() + .doc("elasticsearch host address") + .stringConf + .createWithDefault("127.0.0.1") + + val OAP_ELASTICSEARCH_PORT = + SqlConfAdapter.buildConf("spark.sql.oap.elasticsearch.port") + .internal() + .doc("elasticsearch port") + .intConf + .createWithDefault(9300) } diff --git a/src/main/scala/org/apache/spark/sql/oap/listener/OapListener.scala b/src/main/scala/org/apache/spark/sql/oap/listener/OapListener.scala index 8d70443af66..3f49c025504 100644 --- a/src/main/scala/org/apache/spark/sql/oap/listener/OapListener.scala +++ b/src/main/scala/org/apache/spark/sql/oap/listener/OapListener.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.oap.listener import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} -import org.apache.spark.sql.execution.datasources.oap.filecache.FiberCacheManagerSensor import org.apache.spark.sql.execution.datasources.oap.io.OapIndexInfo import org.apache.spark.sql.oap.OapRuntime @@ -44,9 +43,9 @@ class OapListener extends SparkListener { override def onOtherEvent(event: SparkListenerEvent): Unit = event match { case customInfo: SparkListenerCustomInfoUpdate => if (customInfo.clazzName.contains("OapFiberCacheHeartBeatMessager")) { - OapRuntime.getOrCreate.fiberSensor.update(customInfo) + OapRuntime.getOrCreate.fiberSensor.updateLocations(customInfo) } else if (customInfo.clazzName.contains("FiberCacheManagerMessager")) { - FiberCacheManagerSensor.update(customInfo) + OapRuntime.getOrCreate.fiberSensor.updateMetrics(customInfo) } case indexInfo: SparkListenerOapIndexInfoUpdate => OapIndexInfo.update(indexInfo) diff --git a/src/main/scala/org/apache/spark/sql/oap/rpc/OapRpcManagerSlave.scala b/src/main/scala/org/apache/spark/sql/oap/rpc/OapRpcManagerSlave.scala index 78dcc808590..433182f81ef 100644 --- a/src/main/scala/org/apache/spark/sql/oap/rpc/OapRpcManagerSlave.scala +++ b/src/main/scala/org/apache/spark/sql/oap/rpc/OapRpcManagerSlave.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.execution.datasources.oap.filecache.{CacheStats, Fib import org.apache.spark.sql.execution.datasources.oap.io.OapIndexInfo import org.apache.spark.sql.internal.oap.OapConf import org.apache.spark.sql.oap.OapRuntime +import org.apache.spark.sql.oap.adapter.RpcEndpointRefAdapter import org.apache.spark.sql.oap.rpc.OapMessages._ import org.apache.spark.storage.BlockManager import org.apache.spark.util.{ThreadUtils, Utils} @@ -61,7 +62,8 @@ private[spark] class OapRpcManagerSlave( } private def initialize() = { - driverEndpoint.askWithRetry[Boolean](RegisterOapRpcManager(executorId, slaveEndpoint)) + RpcEndpointRefAdapter.askSync[Boolean]( + driverEndpoint, RegisterOapRpcManager(executorId, slaveEndpoint)) } override private[spark] def send(message: OapMessage): Unit = { diff --git a/src/main/scala/org/apache/spark/sql/oap/ui/FiberCacheManagerPage.scala b/src/main/scala/org/apache/spark/sql/oap/ui/FiberCacheManagerPage.scala index 6ed517198de..0969f6e8f10 100644 --- a/src/main/scala/org/apache/spark/sql/oap/ui/FiberCacheManagerPage.scala +++ b/src/main/scala/org/apache/spark/sql/oap/ui/FiberCacheManagerPage.scala @@ -22,7 +22,8 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.datasources.oap.filecache.{CacheStats, FiberCacheManagerSensor} +import org.apache.spark.sql.execution.datasources.oap.filecache.CacheStats +import org.apache.spark.sql.oap.OapRuntime import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.ui.exec.ExecutorsListener @@ -53,11 +54,9 @@ private[spark] object FiberCacheManagerPage { val hostPort = status.blockManagerId.hostPort val memUsed = status.memUsed val maxMem = status.maxMem - val cacheStats = if (FiberCacheManagerSensor.executorToCacheManager.containsKey(execId)) { - FiberCacheManagerSensor.executorToCacheManager.get(execId) - } else { - CacheStats() - } + val cacheStats = + OapRuntime.getOrCreate.fiberSensor.executorToCacheManager.getOrDefault(execId, CacheStats()) + new FiberCacheManagerSummary( execId, diff --git a/src/main/spark2.1/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/src/main/spark2.1/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 42e15241553..00d54895c42 100644 --- a/src/main/spark2.1/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/src/main/spark2.1/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -172,6 +172,7 @@ indexCol indexType : BTREE | BITMAP + | ES ; unsupportedHiveNativeCommands @@ -956,6 +957,7 @@ SINDICES: 'SINDICES' | 'OINDICES'; BTREE: 'BTREE'; BLOOM: 'BLOOM'; BITMAP: 'BITMAP'; +ES: 'ES'; STRING : '\'' ( ~('\''|'\\') | ('\\' .) )* '\'' diff --git a/src/main/spark2.1/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/src/main/spark2.1/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java new file mode 100644 index 00000000000..cb51cb499ee --- /dev/null +++ b/src/main/spark2.1/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -0,0 +1,549 @@ +/* + * 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.datasources.parquet; + +import java.io.IOException; + +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Dictionary; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.page.*; +import org.apache.parquet.column.values.ValuesReader; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.PrimitiveType; + +import org.apache.spark.sql.execution.vectorized.ColumnVector; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.DecimalType; + +import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL; +import static org.apache.spark.sql.execution.datasources.parquet.SpecificParquetRecordReaderBase.ValuesReaderIntIterator; +import static org.apache.spark.sql.execution.datasources.parquet.SpecificParquetRecordReaderBase.createRLEIterator; + +/** + * Decoder to return values from a single column. + */ +public class VectorizedColumnReader { + /** + * Total number of values read. + */ + private long valuesRead; + + /** + * value that indicates the end of the current page. That is, + * if valuesRead == endOfPageValueCount, we are at the end of the page. + */ + private long endOfPageValueCount; + + /** + * The dictionary, if this column has dictionary encoding. + */ + private final Dictionary dictionary; + + /** + * If true, the current page is dictionary encoded. + */ + private boolean isCurrentPageDictionaryEncoded; + + /** + * Maximum definition level for this column. + */ + private final int maxDefLevel; + + /** + * Repetition/Definition/Value readers. + */ + private SpecificParquetRecordReaderBase.IntIterator repetitionLevelColumn; + private SpecificParquetRecordReaderBase.IntIterator definitionLevelColumn; + private ValuesReader dataColumn; + + // Only set if vectorized decoding is true. This is used instead of the row by row decoding + // with `definitionLevelColumn`. + private VectorizedRleValuesReader defColumn; + + /** + * Total number of values in this column (in this row group). + */ + private final long totalValueCount; + + /** + * Total values in the current page. + */ + private int pageValueCount; + + private final PageReader pageReader; + private final ColumnDescriptor descriptor; + + public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader) + throws IOException { + this.descriptor = descriptor; + this.pageReader = pageReader; + this.maxDefLevel = descriptor.getMaxDefinitionLevel(); + + DictionaryPage dictionaryPage = pageReader.readDictionaryPage(); + if (dictionaryPage != null) { + try { + this.dictionary = dictionaryPage.getEncoding().initDictionary(descriptor, dictionaryPage); + this.isCurrentPageDictionaryEncoded = true; + } catch (IOException e) { + throw new IOException("could not decode the dictionary for " + descriptor, e); + } + } else { + this.dictionary = null; + this.isCurrentPageDictionaryEncoded = false; + } + this.totalValueCount = pageReader.getTotalValueCount(); + if (totalValueCount == 0) { + throw new IOException("totalValueCount == 0"); + } + } + + /** + * Advances to the next value. Returns true if the value is non-null. + */ + private boolean next() throws IOException { + if (valuesRead >= endOfPageValueCount) { + if (valuesRead >= totalValueCount) { + // How do we get here? Throw end of stream exception? + return false; + } + readPage(); + } + ++valuesRead; + // TODO: Don't read for flat schemas + //repetitionLevel = repetitionLevelColumn.nextInt(); + return definitionLevelColumn.nextInt() == maxDefLevel; + } + + /** + * Reads `total` values from this columnReader into column. + */ + void readBatch(int total, ColumnVector column) throws IOException { + int rowId = 0; + ColumnVector dictionaryIds = null; + if (dictionary != null) { + // SPARK-16334: We only maintain a single dictionary per row batch, so that it can be used to + // decode all previous dictionary encoded pages if we ever encounter a non-dictionary encoded + // page. + dictionaryIds = column.reserveDictionaryIds(total); + } + while (total > 0) { + // Compute the number of values we want to read in this page. + int leftInPage = (int) (endOfPageValueCount - valuesRead); + if (leftInPage == 0) { + readPage(); + leftInPage = (int) (endOfPageValueCount - valuesRead); + } + int num = Math.min(total, leftInPage); + if (isCurrentPageDictionaryEncoded) { + // Read and decode dictionary ids. + defColumn.readIntegers( + num, dictionaryIds, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + if (column.hasDictionary() || (rowId == 0 && + (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT32 || + descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT64 || + descriptor.getType() == PrimitiveType.PrimitiveTypeName.FLOAT || + descriptor.getType() == PrimitiveType.PrimitiveTypeName.DOUBLE || + descriptor.getType() == PrimitiveType.PrimitiveTypeName.BINARY))) { + // Column vector supports lazy decoding of dictionary values so just set the dictionary. + // We can't do this if rowId != 0 AND the column doesn't have a dictionary (i.e. some + // non-dictionary encoded values have already been added). + column.setDictionary(dictionary); + } else { + decodeDictionaryIds(rowId, num, column, dictionaryIds); + } + } else { + if (column.hasDictionary() && rowId != 0) { + // This batch already has dictionary encoded values but this new page is not. The batch + // does not support a mix of dictionary and not so we will decode the dictionary. + decodeDictionaryIds(0, rowId, column, column.getDictionaryIds()); + } + column.setDictionary(null); + switch (descriptor.getType()) { + case BOOLEAN: + readBooleanBatch(rowId, num, column); + break; + case INT32: + readIntBatch(rowId, num, column); + break; + case INT64: + readLongBatch(rowId, num, column); + break; + case INT96: + readBinaryBatch(rowId, num, column); + break; + case FLOAT: + readFloatBatch(rowId, num, column); + break; + case DOUBLE: + readDoubleBatch(rowId, num, column); + break; + case BINARY: + readBinaryBatch(rowId, num, column); + break; + case FIXED_LEN_BYTE_ARRAY: + readFixedLenByteArrayBatch(rowId, num, column, descriptor.getTypeLength()); + break; + default: + throw new IOException("Unsupported type: " + descriptor.getType()); + } + } + + valuesRead += num; + rowId += num; + total -= num; + } + } + + /** + * Reads `num` values into column, decoding the values from `dictionaryIds` and `dictionary`. + */ + private void decodeDictionaryIds(int rowId, int num, ColumnVector column, + ColumnVector dictionaryIds) { + switch (descriptor.getType()) { + case INT32: + if (column.dataType() == DataTypes.IntegerType || + DecimalType.is32BitDecimalType(column.dataType())) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.putInt(i, dictionary.decodeToInt(dictionaryIds.getDictId(i))); + } + } + } else if (column.dataType() == DataTypes.ByteType) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.putByte(i, (byte) dictionary.decodeToInt(dictionaryIds.getDictId(i))); + } + } + } else if (column.dataType() == DataTypes.ShortType) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.putShort(i, (short) dictionary.decodeToInt(dictionaryIds.getDictId(i))); + } + } + } else { + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); + } + break; + + case INT64: + if (column.dataType() == DataTypes.LongType || + DecimalType.is64BitDecimalType(column.dataType())) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.putLong(i, dictionary.decodeToLong(dictionaryIds.getDictId(i))); + } + } + } else { + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); + } + break; + + case FLOAT: + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.putFloat(i, dictionary.decodeToFloat(dictionaryIds.getDictId(i))); + } + } + break; + + case DOUBLE: + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.putDouble(i, dictionary.decodeToDouble(dictionaryIds.getDictId(i))); + } + } + break; + case INT96: + if (column.dataType() == DataTypes.TimestampType) { + for (int i = rowId; i < rowId + num; ++i) { + // TODO: Convert dictionary of Binaries to dictionary of Longs + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v)); + } + } + } else { + throw new UnsupportedOperationException(); + } + break; + case BINARY: + // TODO: this is incredibly inefficient as it blows up the dictionary right here. We + // need to do this better. We should probably add the dictionary data to the ColumnVector + // and reuse it across batches. This should mean adding a ByteArray would just update + // the length and offset. + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putByteArray(i, v.getBytes()); + } + } + break; + case FIXED_LEN_BYTE_ARRAY: + // DecimalType written in the legacy mode + if (DecimalType.is32BitDecimalType(column.dataType())) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putInt(i, (int) ParquetRowConverter.binaryToUnscaledLong(v)); + } + } + } else if (DecimalType.is64BitDecimalType(column.dataType())) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putLong(i, ParquetRowConverter.binaryToUnscaledLong(v)); + } + } + } else if (DecimalType.isByteArrayDecimalType(column.dataType())) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putByteArray(i, v.getBytes()); + } + } + } else { + throw new UnsupportedOperationException(); + } + break; + + default: + throw new UnsupportedOperationException("Unsupported type: " + descriptor.getType()); + } + } + + /** + * For all the read*Batch functions, reads `num` values from this columnReader into column. It + * is guaranteed that num is smaller than the number of values left in the current page. + */ + + private void readBooleanBatch(int rowId, int num, ColumnVector column) throws IOException { + assert(column.dataType() == DataTypes.BooleanType); + defColumn.readBooleans( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } + + private void readIntBatch(int rowId, int num, ColumnVector column) throws IOException { + // This is where we implement support for the valid type conversions. + // TODO: implement remaining type conversions + if (column.dataType() == DataTypes.IntegerType || column.dataType() == DataTypes.DateType || + DecimalType.is32BitDecimalType(column.dataType())) { + defColumn.readIntegers( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } else if (column.dataType() == DataTypes.ByteType) { + defColumn.readBytes( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } else if (column.dataType() == DataTypes.ShortType) { + defColumn.readShorts( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } else { + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); + } + } + + private void readLongBatch(int rowId, int num, ColumnVector column) throws IOException { + // This is where we implement support for the valid type conversions. + if (column.dataType() == DataTypes.LongType || + DecimalType.is64BitDecimalType(column.dataType())) { + defColumn.readLongs( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } else { + throw new UnsupportedOperationException("Unsupported conversion to: " + column.dataType()); + } + } + + private void readFloatBatch(int rowId, int num, ColumnVector column) throws IOException { + // This is where we implement support for the valid type conversions. + // TODO: support implicit cast to double? + if (column.dataType() == DataTypes.FloatType) { + defColumn.readFloats( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } else { + throw new UnsupportedOperationException("Unsupported conversion to: " + column.dataType()); + } + } + + private void readDoubleBatch(int rowId, int num, ColumnVector column) throws IOException { + // This is where we implement support for the valid type conversions. + // TODO: implement remaining type conversions + if (column.dataType() == DataTypes.DoubleType) { + defColumn.readDoubles( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } else { + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); + } + } + + private void readBinaryBatch(int rowId, int num, ColumnVector column) throws IOException { + // This is where we implement support for the valid type conversions. + // TODO: implement remaining type conversions + VectorizedValuesReader data = (VectorizedValuesReader) dataColumn; + if (column.isArray()) { + defColumn.readBinarys(num, column, rowId, maxDefLevel, data); + } else if (column.dataType() == DataTypes.TimestampType) { + for (int i = 0; i < num; i++) { + if (defColumn.readInteger() == maxDefLevel) { + column.putLong(rowId + i, + // Read 12 bytes for INT96 + ParquetRowConverter.binaryToSQLTimestamp(data.readBinary(12))); + } else { + column.putNull(rowId + i); + } + } + } else { + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); + } + } + + private void readFixedLenByteArrayBatch(int rowId, int num, + ColumnVector column, int arrayLen) throws IOException { + VectorizedValuesReader data = (VectorizedValuesReader) dataColumn; + // This is where we implement support for the valid type conversions. + // TODO: implement remaining type conversions + if (DecimalType.is32BitDecimalType(column.dataType())) { + for (int i = 0; i < num; i++) { + if (defColumn.readInteger() == maxDefLevel) { + column.putInt(rowId + i, + (int) ParquetRowConverter.binaryToUnscaledLong(data.readBinary(arrayLen))); + } else { + column.putNull(rowId + i); + } + } + } else if (DecimalType.is64BitDecimalType(column.dataType())) { + for (int i = 0; i < num; i++) { + if (defColumn.readInteger() == maxDefLevel) { + column.putLong(rowId + i, + ParquetRowConverter.binaryToUnscaledLong(data.readBinary(arrayLen))); + } else { + column.putNull(rowId + i); + } + } + } else if (DecimalType.isByteArrayDecimalType(column.dataType())) { + for (int i = 0; i < num; i++) { + if (defColumn.readInteger() == maxDefLevel) { + column.putByteArray(rowId + i, data.readBinary(arrayLen).getBytes()); + } else { + column.putNull(rowId + i); + } + } + } else { + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); + } + } + + private void readPage() throws IOException { + DataPage page = pageReader.readPage(); + // TODO: Why is this a visitor? + page.accept(new DataPage.Visitor() { + @Override + public Void visit(DataPageV1 dataPageV1) { + try { + readPageV1(dataPageV1); + return null; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public Void visit(DataPageV2 dataPageV2) { + try { + readPageV2(dataPageV2); + return null; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }); + } + + private void initDataReader(Encoding dataEncoding, byte[] bytes, int offset) throws IOException { + this.endOfPageValueCount = valuesRead + pageValueCount; + if (dataEncoding.usesDictionary()) { + this.dataColumn = null; + if (dictionary == null) { + throw new IOException( + "could not read page in col " + descriptor + + " as the dictionary was missing for encoding " + dataEncoding); + } + @SuppressWarnings("deprecation") + Encoding plainDict = Encoding.PLAIN_DICTIONARY; // var to allow warning suppression + if (dataEncoding != plainDict && dataEncoding != Encoding.RLE_DICTIONARY) { + throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding); + } + this.dataColumn = new VectorizedRleValuesReader(); + this.isCurrentPageDictionaryEncoded = true; + } else { + if (dataEncoding != Encoding.PLAIN) { + throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding); + } + this.dataColumn = new VectorizedPlainValuesReader(); + this.isCurrentPageDictionaryEncoded = false; + } + + try { + dataColumn.initFromPage(pageValueCount, bytes, offset); + } catch (IOException e) { + throw new IOException("could not read page in col " + descriptor, e); + } + } + + private void readPageV1(DataPageV1 page) throws IOException { + this.pageValueCount = page.getValueCount(); + ValuesReader rlReader = page.getRlEncoding().getValuesReader(descriptor, REPETITION_LEVEL); + ValuesReader dlReader; + + // Initialize the decoders. + if (page.getDlEncoding() != Encoding.RLE && descriptor.getMaxDefinitionLevel() != 0) { + throw new UnsupportedOperationException("Unsupported encoding: " + page.getDlEncoding()); + } + int bitWidth = BytesUtils.getWidthFromMaxInt(descriptor.getMaxDefinitionLevel()); + this.defColumn = new VectorizedRleValuesReader(bitWidth); + dlReader = this.defColumn; + this.repetitionLevelColumn = new ValuesReaderIntIterator(rlReader); + this.definitionLevelColumn = new ValuesReaderIntIterator(dlReader); + try { + byte[] bytes = page.getBytes().toByteArray(); + rlReader.initFromPage(pageValueCount, bytes, 0); + int next = rlReader.getNextOffset(); + dlReader.initFromPage(pageValueCount, bytes, next); + next = dlReader.getNextOffset(); + initDataReader(page.getValueEncoding(), bytes, next); + } catch (IOException e) { + throw new IOException("could not read page " + page + " in col " + descriptor, e); + } + } + + private void readPageV2(DataPageV2 page) throws IOException { + this.pageValueCount = page.getValueCount(); + this.repetitionLevelColumn = createRLEIterator(descriptor.getMaxRepetitionLevel(), + page.getRepetitionLevels(), descriptor); + + int bitWidth = BytesUtils.getWidthFromMaxInt(descriptor.getMaxDefinitionLevel()); + this.defColumn = new VectorizedRleValuesReader(bitWidth); + this.definitionLevelColumn = new ValuesReaderIntIterator(this.defColumn); + this.defColumn.initFromBuffer( + this.pageValueCount, page.getDefinitionLevels().toByteArray()); + try { + initDataReader(page.getDataEncoding(), page.getData().toByteArray(), 0); + } catch (IOException e) { + throw new IOException("could not read page " + page + " in col " + descriptor, e); + } + } +} diff --git a/src/main/spark2.1/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java b/src/main/spark2.1/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java new file mode 100644 index 00000000000..98018b7f48b --- /dev/null +++ b/src/main/spark2.1/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java @@ -0,0 +1,177 @@ +/* + * 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.datasources.parquet; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +import org.apache.spark.sql.execution.vectorized.ColumnVector; +import org.apache.spark.unsafe.Platform; + +import org.apache.parquet.column.values.ValuesReader; +import org.apache.parquet.io.api.Binary; + +/** + * An implementation of the Parquet PLAIN decoder that supports the vectorized interface. + */ +public class VectorizedPlainValuesReader extends ValuesReader implements VectorizedValuesReader { + private byte[] buffer; + private int offset; + private int bitOffset; // Only used for booleans. + private ByteBuffer byteBuffer; // used to wrap the byte array buffer + + private static final boolean bigEndianPlatform = + ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); + + public VectorizedPlainValuesReader() { + } + + @Override + public void initFromPage(int valueCount, byte[] bytes, int offset) throws IOException { + this.buffer = bytes; + this.offset = offset + Platform.BYTE_ARRAY_OFFSET; + if (bigEndianPlatform) { + byteBuffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + } + } + + @Override + public void skip() { + throw new UnsupportedOperationException(); + } + + @Override + public final void readBooleans(int total, ColumnVector c, int rowId) { + // TODO: properly vectorize this + for (int i = 0; i < total; i++) { + c.putBoolean(rowId + i, readBoolean()); + } + } + + @Override + public final void readIntegers(int total, ColumnVector c, int rowId) { + c.putIntsLittleEndian(rowId, total, buffer, offset - Platform.BYTE_ARRAY_OFFSET); + offset += 4 * total; + } + + @Override + public final void readLongs(int total, ColumnVector c, int rowId) { + c.putLongsLittleEndian(rowId, total, buffer, offset - Platform.BYTE_ARRAY_OFFSET); + offset += 8 * total; + } + + @Override + public final void readFloats(int total, ColumnVector c, int rowId) { + c.putFloats(rowId, total, buffer, offset - Platform.BYTE_ARRAY_OFFSET); + offset += 4 * total; + } + + @Override + public final void readDoubles(int total, ColumnVector c, int rowId) { + c.putDoubles(rowId, total, buffer, offset - Platform.BYTE_ARRAY_OFFSET); + offset += 8 * total; + } + + @Override + public final void readBytes(int total, ColumnVector c, int rowId) { + for (int i = 0; i < total; i++) { + // Bytes are stored as a 4-byte little endian int. Just read the first byte. + // TODO: consider pushing this in ColumnVector by adding a readBytes with a stride. + c.putByte(rowId + i, Platform.getByte(buffer, offset)); + offset += 4; + } + } + + @Override + public final boolean readBoolean() { + byte b = Platform.getByte(buffer, offset); + boolean v = (b & (1 << bitOffset)) != 0; + bitOffset += 1; + if (bitOffset == 8) { + bitOffset = 0; + offset++; + } + return v; + } + + @Override + public final int readInteger() { + int v = Platform.getInt(buffer, offset); + if (bigEndianPlatform) { + v = java.lang.Integer.reverseBytes(v); + } + offset += 4; + return v; + } + + @Override + public final long readLong() { + long v = Platform.getLong(buffer, offset); + if (bigEndianPlatform) { + v = java.lang.Long.reverseBytes(v); + } + offset += 8; + return v; + } + + @Override + public final byte readByte() { + return (byte)readInteger(); + } + + @Override + public final float readFloat() { + float v; + if (!bigEndianPlatform) { + v = Platform.getFloat(buffer, offset); + } else { + v = byteBuffer.getFloat(offset - Platform.BYTE_ARRAY_OFFSET); + } + offset += 4; + return v; + } + + @Override + public final double readDouble() { + double v; + if (!bigEndianPlatform) { + v = Platform.getDouble(buffer, offset); + } else { + v = byteBuffer.getDouble(offset - Platform.BYTE_ARRAY_OFFSET); + } + offset += 8; + return v; + } + + @Override + public final void readBinary(int total, ColumnVector v, int rowId) { + for (int i = 0; i < total; i++) { + int len = readInteger(); + int start = offset; + offset += len; + v.putByteArray(rowId + i, buffer, start - Platform.BYTE_ARRAY_OFFSET, len); + } + } + + @Override + public final Binary readBinary(int len) { + Binary result = Binary.fromConstantByteArray(buffer, offset - Platform.BYTE_ARRAY_OFFSET, len); + offset += len; + return result; + } +} diff --git a/src/main/spark2.1/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedRleValuesReader.java b/src/main/spark2.1/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedRleValuesReader.java new file mode 100644 index 00000000000..62157389013 --- /dev/null +++ b/src/main/spark2.1/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedRleValuesReader.java @@ -0,0 +1,613 @@ +/* + * 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.datasources.parquet; + +import org.apache.parquet.Preconditions; +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.values.ValuesReader; +import org.apache.parquet.column.values.bitpacking.BytePacker; +import org.apache.parquet.column.values.bitpacking.Packer; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.io.api.Binary; + +import org.apache.spark.sql.execution.vectorized.ColumnVector; + +/** + * A values reader for Parquet's run-length encoded data. This is based off of the version in + * parquet-mr with these changes: + * - Supports the vectorized interface. + * - Works on byte arrays(byte[]) instead of making byte streams. + * + * This encoding is used in multiple places: + * - Definition/Repetition levels + * - Dictionary ids. + */ +public final class VectorizedRleValuesReader extends ValuesReader + implements VectorizedValuesReader { + // Current decoding mode. The encoded data contains groups of either run length encoded data + // (RLE) or bit packed data. Each group contains a header that indicates which group it is and + // the number of values in the group. + // More details here: https://github.com/Parquet/parquet-format/blob/master/Encodings.md + private enum MODE { + RLE, + PACKED + } + + // Encoded data. + private byte[] in; + private int end; + private int offset; + + // bit/byte width of decoded data and utility to batch unpack them. + private int bitWidth; + private int bytesWidth; + private BytePacker packer; + + // Current decoding mode and values + private MODE mode; + private int currentCount; + private int currentValue; + + // Buffer of decoded values if the values are PACKED. + private int[] currentBuffer = new int[16]; + private int currentBufferIdx = 0; + + // If true, the bit width is fixed. This decoder is used in different places and this also + // controls if we need to read the bitwidth from the beginning of the data stream. + private final boolean fixedWidth; + + public VectorizedRleValuesReader() { + fixedWidth = false; + } + + public VectorizedRleValuesReader(int bitWidth) { + fixedWidth = true; + init(bitWidth); + } + + @Override + public void initFromPage(int valueCount, byte[] page, int start) { + this.offset = start; + this.in = page; + if (fixedWidth) { + if (bitWidth != 0) { + int length = readIntLittleEndian(); + this.end = this.offset + length; + } + } else { + this.end = page.length; + if (this.end != this.offset) init(page[this.offset++] & 255); + } + if (bitWidth == 0) { + // 0 bit width, treat this as an RLE run of valueCount number of 0's. + this.mode = MODE.RLE; + this.currentCount = valueCount; + this.currentValue = 0; + } else { + this.currentCount = 0; + } + } + + // Initialize the reader from a buffer. This is used for the V2 page encoding where the + // definition are in its own buffer. + public void initFromBuffer(int valueCount, byte[] data) { + this.offset = 0; + this.in = data; + this.end = data.length; + if (bitWidth == 0) { + // 0 bit width, treat this as an RLE run of valueCount number of 0's. + this.mode = MODE.RLE; + this.currentCount = valueCount; + this.currentValue = 0; + } else { + this.currentCount = 0; + } + } + + /** + * Initializes the internal state for decoding ints of `bitWidth`. + */ + private void init(int bitWidth) { + Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32"); + this.bitWidth = bitWidth; + this.bytesWidth = BytesUtils.paddedByteCountFromBits(bitWidth); + this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth); + } + + @Override + public int getNextOffset() { + return this.end; + } + + @Override + public boolean readBoolean() { + return this.readInteger() != 0; + } + + @Override + public void skip() { + this.readInteger(); + } + + @Override + public int readValueDictionaryId() { + return readInteger(); + } + + @Override + public int readInteger() { + if (this.currentCount == 0) { this.readNextGroup(); } + + this.currentCount--; + switch (mode) { + case RLE: + return this.currentValue; + case PACKED: + return this.currentBuffer[currentBufferIdx++]; + } + throw new RuntimeException("Unreachable"); + } + + /** + * Reads `total` ints into `c` filling them in starting at `c[rowId]`. This reader + * reads the definition levels and then will read from `data` for the non-null values. + * If the value is null, c will be populated with `nullValue`. Note that `nullValue` is only + * necessary for readIntegers because we also use it to decode dictionaryIds and want to make + * sure it always has a value in range. + * + * This is a batched version of this logic: + * if (this.readInt() == level) { + * c[rowId] = data.readInteger(); + * } else { + * c[rowId] = null; + * } + */ + public void readIntegers(int total, ColumnVector c, int rowId, int level, + VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readIntegers(n, c, rowId); + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + c.putInt(rowId + i, data.readInteger()); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + // TODO: can this code duplication be removed without a perf penalty? + public void readBooleans(int total, ColumnVector c, + int rowId, int level, VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readBooleans(n, c, rowId); + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + c.putBoolean(rowId + i, data.readBoolean()); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + public void readBytes(int total, ColumnVector c, + int rowId, int level, VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readBytes(n, c, rowId); + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + c.putByte(rowId + i, data.readByte()); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + public void readShorts(int total, ColumnVector c, + int rowId, int level, VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + for (int i = 0; i < n; i++) { + c.putShort(rowId + i, (short)data.readInteger()); + } + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + c.putShort(rowId + i, (short)data.readInteger()); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + public void readLongs(int total, ColumnVector c, int rowId, int level, + VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readLongs(n, c, rowId); + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + c.putLong(rowId + i, data.readLong()); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + public void readFloats(int total, ColumnVector c, int rowId, int level, + VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readFloats(n, c, rowId); + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + c.putFloat(rowId + i, data.readFloat()); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + public void readDoubles(int total, ColumnVector c, int rowId, int level, + VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readDoubles(n, c, rowId); + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + c.putDouble(rowId + i, data.readDouble()); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + public void readBinarys(int total, ColumnVector c, int rowId, int level, + VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readBinary(n, c, rowId); + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + data.readBinary(1, c, rowId + i); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + /** + * Decoding for dictionary ids. The IDs are populated into `values` and the nullability is + * populated into `nulls`. + */ + public void readIntegers(int total, ColumnVector values, ColumnVector nulls, int rowId, int level, + VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readIntegers(n, values, rowId); + } else { + nulls.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + values.putInt(rowId + i, data.readInteger()); + } else { + nulls.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + + // The RLE reader implements the vectorized decoding interface when used to decode dictionary + // IDs. This is different than the above APIs that decodes definitions levels along with values. + // Since this is only used to decode dictionary IDs, only decoding integers is supported. + @Override + public void readIntegers(int total, ColumnVector c, int rowId) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + c.putInts(rowId, n, currentValue); + break; + case PACKED: + c.putInts(rowId, n, currentBuffer, currentBufferIdx); + currentBufferIdx += n; + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + @Override + public byte readByte() { + throw new UnsupportedOperationException("only readInts is valid."); + } + + @Override + public void readBytes(int total, ColumnVector c, int rowId) { + throw new UnsupportedOperationException("only readInts is valid."); + } + + @Override + public void readLongs(int total, ColumnVector c, int rowId) { + throw new UnsupportedOperationException("only readInts is valid."); + } + + @Override + public void readBinary(int total, ColumnVector c, int rowId) { + throw new UnsupportedOperationException("only readInts is valid."); + } + + @Override + public void readBooleans(int total, ColumnVector c, int rowId) { + throw new UnsupportedOperationException("only readInts is valid."); + } + + @Override + public void readFloats(int total, ColumnVector c, int rowId) { + throw new UnsupportedOperationException("only readInts is valid."); + } + + @Override + public void readDoubles(int total, ColumnVector c, int rowId) { + throw new UnsupportedOperationException("only readInts is valid."); + } + + @Override + public Binary readBinary(int len) { + throw new UnsupportedOperationException("only readInts is valid."); + } + + /** + * Reads the next varint encoded int. + */ + private int readUnsignedVarInt() { + int value = 0; + int shift = 0; + int b; + do { + b = in[offset++] & 255; + value |= (b & 0x7F) << shift; + shift += 7; + } while ((b & 0x80) != 0); + return value; + } + + /** + * Reads the next 4 byte little endian int. + */ + private int readIntLittleEndian() { + int ch4 = in[offset] & 255; + int ch3 = in[offset + 1] & 255; + int ch2 = in[offset + 2] & 255; + int ch1 = in[offset + 3] & 255; + offset += 4; + return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0)); + } + + /** + * Reads the next byteWidth little endian int. + */ + private int readIntLittleEndianPaddedOnBitWidth() { + switch (bytesWidth) { + case 0: + return 0; + case 1: + return in[offset++] & 255; + case 2: { + int ch2 = in[offset] & 255; + int ch1 = in[offset + 1] & 255; + offset += 2; + return (ch1 << 8) + ch2; + } + case 3: { + int ch3 = in[offset] & 255; + int ch2 = in[offset + 1] & 255; + int ch1 = in[offset + 2] & 255; + offset += 3; + return (ch1 << 16) + (ch2 << 8) + (ch3 << 0); + } + case 4: { + return readIntLittleEndian(); + } + } + throw new RuntimeException("Unreachable"); + } + + private int ceil8(int value) { + return (value + 7) / 8; + } + + /** + * Reads the next group. + */ + private void readNextGroup() { + int header = readUnsignedVarInt(); + this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED; + switch (mode) { + case RLE: + this.currentCount = header >>> 1; + this.currentValue = readIntLittleEndianPaddedOnBitWidth(); + return; + case PACKED: + int numGroups = header >>> 1; + this.currentCount = numGroups * 8; + int bytesToRead = ceil8(this.currentCount * this.bitWidth); + + if (this.currentBuffer.length < this.currentCount) { + this.currentBuffer = new int[this.currentCount]; + } + currentBufferIdx = 0; + int valueIndex = 0; + for (int byteIndex = offset; valueIndex < this.currentCount; byteIndex += this.bitWidth) { + this.packer.unpack8Values(in, byteIndex, this.currentBuffer, valueIndex); + valueIndex += 8; + } + offset += bytesToRead; + return; + default: + throw new ParquetDecodingException("not a valid mode " + this.mode); + } + } +} diff --git a/src/main/spark2.1/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/src/main/spark2.1/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index e7dd53f467c..22f43840f0e 100644 --- a/src/main/spark2.1/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/src/main/spark2.1/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -503,7 +503,7 @@ case class FileSourceScanExec( } } else { val cachedHosts = OapRuntime.getOrCreate.fiberSensor.getHosts(file.getPath.toString) - val hosts = cachedHosts.toBuffer ++ getBlockHosts(blockLocations, 0, file.getLen) + val hosts = cachedHosts ++ getBlockHosts(blockLocations, 0, file.getLen) Seq(PartitionedFile( partition.values, file.getPath.toUri.toString, 0, file.getLen, hosts.toArray)) } diff --git a/src/main/spark2.1/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/src/main/spark2.1/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 1cb1f78390d..543ecc297ab 100644 --- a/src/main/spark2.1/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/src/main/spark2.1/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -1405,7 +1405,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * * {{{ * CREATE OINDEX [IF NOT EXISTS] indexName ON tableName (col1 [ASC | DESC], col2, ...) - * [USING (BTREE | BITMAP)] [PARTITION (partcol1=val1, partcol2=val2 ...)] + * [USING (BTREE | BITMAP | ES)] [PARTITION (partcol1=val1, partcol2=val2 ...)] * }}} */ override def visitOapCreateIndex(ctx: OapCreateIndexContext): LogicalPlan = @@ -1447,8 +1447,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { withOrigin(ctx) { if (ctx.BTREE != null) { BTreeIndexType - } else { + } else if (ctx.BITMAP != null) { BitMapIndexType + } else { + ESIndexType } } } diff --git a/src/main/spark2.1/scala/org/apache/spark/sql/execution/datasources/oap/OapOutputWriter.scala b/src/main/spark2.1/scala/org/apache/spark/sql/execution/datasources/oap/OapOutputWriter.scala new file mode 100644 index 00000000000..74a9014267f --- /dev/null +++ b/src/main/spark2.1/scala/org/apache/spark/sql/execution/datasources/oap/OapOutputWriter.scala @@ -0,0 +1,68 @@ +/* + * 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.datasources.oap + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.{OutputWriter, WriteResult} +import org.apache.spark.sql.execution.datasources.oap.OapWriteResult +import org.apache.spark.sql.execution.datasources.oap.io.OapDataWriter +import org.apache.spark.sql.types.StructType + + +private[oap] class OapOutputWriter( + path: String, + dataSchema: StructType, + context: TaskAttemptContext) extends OutputWriter { + private var rowCount = 0 + private var partitionString: String = "" + + override def setPartitionString(ps: String): Unit = { + partitionString = ps + } + + private val writer: OapDataWriter = { + val isCompressed = FileOutputFormat.getCompressOutput(context) + val conf = context.getConfiguration + val file: Path = new Path(path) + val fs = file.getFileSystem(conf) + val fileOut = fs.create(file, false) + + new OapDataWriter(isCompressed, fileOut, dataSchema, conf) + } + + override def write(row: Row): Unit = throw new NotImplementedError("write(row: Row)") + + override protected[sql] def writeInternal(row: InternalRow): Unit = { + rowCount += 1 + writer.write(row) + } + + override def close(): Unit = { + writer.close() + } + + override def writeStatus(): WriteResult = { + OapWriteResult(dataFileName, rowCount, partitionString) + } + + def dataFileName: String = new Path(path).getName +} diff --git a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexOutputWriter.scala b/src/main/spark2.1/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexOutputWriter.scala similarity index 92% rename from src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexOutputWriter.scala rename to src/main/spark2.1/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexOutputWriter.scala index 43f3fc38478..25f05d6b904 100644 --- a/src/main/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexOutputWriter.scala +++ b/src/main/spark2.1/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexOutputWriter.scala @@ -22,10 +22,10 @@ import org.apache.hadoop.mapreduce.{RecordWriter, TaskAttemptContext} import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.parquet.hadoop.util.ContextUtil -import org.apache.spark.rdd.InputFileNameHolder import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.OutputWriter +import org.apache.spark.sql.oap.adapter.InputFileNameHolderAdapter // TODO: parameter name "path" is ambiguous private[index] class OapIndexOutputWriter( @@ -67,7 +67,7 @@ private[index] class OapIndexOutputWriter( } private def initWriter(): Unit = { - inputFileName = InputFileNameHolder.getInputFileName().toString + inputFileName = InputFileNameHolderAdapter.getInputFileName().toString recordWriter = outputFormat.getRecordWriter(context) rowCount = 0 } @@ -82,7 +82,7 @@ private[index] class OapIndexOutputWriter( } private def checkStartOfNewFile(): Unit = { - if (inputFileName != InputFileNameHolder.getInputFileName().toString) { + if (inputFileName != InputFileNameHolderAdapter.getInputFileName().toString) { closeWriter() initWriter() } diff --git a/src/main/scala/org/apache/spark/sql/hive/OapSessionState.scala b/src/main/spark2.1/scala/org/apache/spark/sql/hive/OapSessionState.scala similarity index 100% rename from src/main/scala/org/apache/spark/sql/hive/OapSessionState.scala rename to src/main/spark2.1/scala/org/apache/spark/sql/hive/OapSessionState.scala diff --git a/src/main/scala/org/apache/spark/sql/hive/thriftserver/OapEnv.scala b/src/main/spark2.1/scala/org/apache/spark/sql/hive/thriftserver/OapEnv.scala similarity index 95% rename from src/main/scala/org/apache/spark/sql/hive/thriftserver/OapEnv.scala rename to src/main/spark2.1/scala/org/apache/spark/sql/hive/thriftserver/OapEnv.scala index c0e92bf8390..82a0df8c640 100644 --- a/src/main/scala/org/apache/spark/sql/hive/thriftserver/OapEnv.scala +++ b/src/main/spark2.1/scala/org/apache/spark/sql/hive/thriftserver/OapEnv.scala @@ -28,6 +28,10 @@ import org.apache.spark.sql.oap.listener.OapListener import org.apache.spark.sql.oap.ui.OapTab import org.apache.spark.util.Utils +/** + * Most of the code in init() are copied from SparkSQLEnv. Please include code from the + * corresponding Spark version. + */ private[hive] object OapEnv extends Logging { logDebug("Initializing Oap Env") diff --git a/src/main/scala/org/apache/spark/sql/oap/OapSession.scala b/src/main/spark2.1/scala/org/apache/spark/sql/oap/OapSession.scala similarity index 100% rename from src/main/scala/org/apache/spark/sql/oap/OapSession.scala rename to src/main/spark2.1/scala/org/apache/spark/sql/oap/OapSession.scala diff --git a/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/AggregateFunctionAdapter.scala b/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/AggregateFunctionAdapter.scala new file mode 100644 index 00000000000..fc22340330f --- /dev/null +++ b/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/AggregateFunctionAdapter.scala @@ -0,0 +1,24 @@ +/* + * 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.oap.adapter + +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction + +object AggregateFunctionAdapter { + def supportsPartial(func: AggregateFunction): Boolean = func.supportsPartial +} diff --git a/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/FileIndexAdapter.scala b/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/FileIndexAdapter.scala new file mode 100644 index 00000000000..303311f1f5d --- /dev/null +++ b/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/FileIndexAdapter.scala @@ -0,0 +1,33 @@ +/* + * 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.oap.adapter + +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.execution.datasources.{FileIndex, PartitionDirectory} + +object FileIndexAdapter { + /** + * Parameter dataFilters is added in Spark 2.2 and later. In 2.1, we just ignore it. + */ + def listFiles( + fileIndex: FileIndex, + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { + fileIndex.listFiles(partitionFilters) + } +} diff --git a/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/FileSourceScanExecAdapter.scala b/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/FileSourceScanExecAdapter.scala new file mode 100644 index 00000000000..a88ef44b788 --- /dev/null +++ b/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/FileSourceScanExecAdapter.scala @@ -0,0 +1,45 @@ +/* + * 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.oap.adapter + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute} +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation} +import org.apache.spark.sql.types.StructType + +object FileSourceScanExecAdapter extends Logging{ + def createFileSourceScanExec( + relation: HadoopFsRelation, + output: Seq[Attribute], + requiredSchema: StructType, + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression], + metastoreTableIdentifier: Option[TableIdentifier]): FileSourceScanExec = { + val pushedDownFilters = dataFilters.flatMap(DataSourceStrategy.translateFilter) + logInfo(s"Pushed Filters: ${pushedDownFilters.mkString(",")}") + FileSourceScanExec( + relation, + output, + requiredSchema, + partitionFilters, + pushedDownFilters, + metastoreTableIdentifier) + } +} diff --git a/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/InputFileNameHolderAdapter.scala b/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/InputFileNameHolderAdapter.scala new file mode 100644 index 00000000000..df8bb22eff5 --- /dev/null +++ b/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/InputFileNameHolderAdapter.scala @@ -0,0 +1,28 @@ +/* + * 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.oap.adapter + +import org.apache.spark.rdd.InputFileNameHolder +import org.apache.spark.unsafe.types.UTF8String + +object InputFileNameHolderAdapter { + /** + * Returns the holding file name or empty string if it is unknown. + */ + def getInputFileName(): UTF8String = InputFileNameHolder.getInputFileName() +} diff --git a/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/LogicalPlanAdapter.scala b/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/LogicalPlanAdapter.scala new file mode 100644 index 00000000000..2da7c2ea80d --- /dev/null +++ b/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/LogicalPlanAdapter.scala @@ -0,0 +1,25 @@ +/* + * 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.oap.adapter + +import org.apache.spark.sql.catalyst.plans.logical.{Statistics, LogicalPlan} +import org.apache.spark.sql.internal.SQLConf + +object LogicalPlanAdapter { + def getStatistics(plan: LogicalPlan, conf:SQLConf): Statistics = plan.statistics +} diff --git a/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/RpcEndpointRefAdapter.scala b/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/RpcEndpointRefAdapter.scala new file mode 100644 index 00000000000..564bae4f879 --- /dev/null +++ b/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/RpcEndpointRefAdapter.scala @@ -0,0 +1,28 @@ +/* + * 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.oap.adapter + +import scala.reflect.ClassTag + +import org.apache.spark.rpc.RpcEndpointRef + +object RpcEndpointRefAdapter { + def askSync[T: ClassTag](endpoint: RpcEndpointRef, message: Any): T = { + endpoint.askWithRetry(message) + } +} diff --git a/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/SqlConfAdapter.scala b/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/SqlConfAdapter.scala new file mode 100644 index 00000000000..79facf5c7fa --- /dev/null +++ b/src/main/spark2.1/scala/org/apache/spark/sql/oap/adapter/SqlConfAdapter.scala @@ -0,0 +1,25 @@ +/* + * 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.oap.adapter + +import org.apache.spark.internal.config.ConfigBuilder +import org.apache.spark.sql.internal.SQLConf.SQLConfigBuilder + +object SqlConfAdapter { + def buildConf(key: String): ConfigBuilder = SQLConfigBuilder(key) +} diff --git a/src/main/spark2.2/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/src/main/spark2.2/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 499f27f00e7..7012c66cdfb 100644 --- a/src/main/spark2.2/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/src/main/spark2.2/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -149,6 +149,7 @@ statement | (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)? tableIdentifier partitionSpec? describeColName? #describeTable | REFRESH TABLE tableIdentifier #refreshTable + | REFRESH SINDEX ON tableIdentifier partitionSpec? #oapRefreshIndices | REFRESH .*? #refreshResource | CACHE LAZY? TABLE tableIdentifier (AS? query)? #cacheTable | UNCACHE TABLE (IF EXISTS)? tableIdentifier #uncacheTable @@ -161,9 +162,32 @@ statement | SET ROLE .*? #failNativeCommand | SET .*? #setConfiguration | RESET #resetConfiguration + | CREATE SINDEX (IF NOT EXISTS)? IDENTIFIER ON + tableIdentifier indexCols (USING indexType)? + partitionSpec? #oapCreateIndex + | DROP SINDEX (IF EXISTS)? IDENTIFIER ON tableIdentifier + partitionSpec? #oapDropIndex + | DISABLE SINDEX IDENTIFIER #oapDisableIndex + | ENABLE SINDEX IDENTIFIER #oapEnableIndex + | SHOW SINDEX (FROM | IN) tableIdentifier #oapShowIndex + | CHECK SINDEX ON tableIdentifier partitionSpec? #oapCheckIndex | unsupportedHiveNativeCommands .*? #failNativeCommand ; +indexCols + : '(' indexCol (',' indexCol)* ')' + ; + +indexCol + : identifier (ASC | DESC)? + ; + +indexType + : BTREE + | BITMAP + | ES + ; + unsupportedHiveNativeCommands : kw1=CREATE kw2=ROLE | kw1=DROP kw2=ROLE @@ -840,6 +864,8 @@ COMMIT: 'COMMIT'; ROLLBACK: 'ROLLBACK'; MACRO: 'MACRO'; IGNORE: 'IGNORE'; +DISABLE: 'DISABLE'; +ENABLE: 'ENABLE'; IF: 'IF'; @@ -958,6 +984,14 @@ INPATH: 'INPATH'; CURRENT_DATE: 'CURRENT_DATE'; CURRENT_TIMESTAMP: 'CURRENT_TIMESTAMP'; +CHECK: 'CHECK'; +SINDEX: 'SINDEX' | 'OINDEX'; +SINDICES: 'SINDICES' | 'OINDICES'; +BTREE: 'BTREE'; +BLOOM: 'BLOOM'; +BITMAP: 'BITMAP'; +ES: 'ES'; + STRING : '\'' ( ~('\''|'\\') | ('\\' .) )* '\'' | '\"' ( ~('\"'|'\\') | ('\\' .) )* '\"' diff --git a/src/main/spark2.2/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/src/main/spark2.2/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java new file mode 100644 index 00000000000..9d641b52872 --- /dev/null +++ b/src/main/spark2.2/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -0,0 +1,570 @@ +/* + * 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.datasources.parquet; + +import java.io.IOException; + +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Dictionary; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.page.*; +import org.apache.parquet.column.values.ValuesReader; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.PrimitiveType; + +import org.apache.spark.sql.catalyst.util.DateTimeUtils; +import org.apache.spark.sql.execution.vectorized.ColumnVector; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.DecimalType; + +import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL; +import static org.apache.spark.sql.execution.datasources.parquet.SpecificParquetRecordReaderBase.ValuesReaderIntIterator; +import static org.apache.spark.sql.execution.datasources.parquet.SpecificParquetRecordReaderBase.createRLEIterator; + +/** + * Decoder to return values from a single column. + */ +public class VectorizedColumnReader { + /** + * Total number of values read. + */ + private long valuesRead; + + /** + * value that indicates the end of the current page. That is, + * if valuesRead == endOfPageValueCount, we are at the end of the page. + */ + private long endOfPageValueCount; + + /** + * The dictionary, if this column has dictionary encoding. + */ + private final Dictionary dictionary; + + /** + * If true, the current page is dictionary encoded. + */ + private boolean isCurrentPageDictionaryEncoded; + + /** + * Maximum definition level for this column. + */ + private final int maxDefLevel; + + /** + * Repetition/Definition/Value readers. + */ + private SpecificParquetRecordReaderBase.IntIterator repetitionLevelColumn; + private SpecificParquetRecordReaderBase.IntIterator definitionLevelColumn; + private ValuesReader dataColumn; + + // Only set if vectorized decoding is true. This is used instead of the row by row decoding + // with `definitionLevelColumn`. + private VectorizedRleValuesReader defColumn; + + /** + * Total number of values in this column (in this row group). + */ + private final long totalValueCount; + + /** + * Total values in the current page. + */ + private int pageValueCount; + + private final PageReader pageReader; + private final ColumnDescriptor descriptor; + + public VectorizedColumnReader(ColumnDescriptor descriptor, PageReader pageReader) + throws IOException { + this.descriptor = descriptor; + this.pageReader = pageReader; + this.maxDefLevel = descriptor.getMaxDefinitionLevel(); + + DictionaryPage dictionaryPage = pageReader.readDictionaryPage(); + if (dictionaryPage != null) { + try { + this.dictionary = dictionaryPage.getEncoding().initDictionary(descriptor, dictionaryPage); + this.isCurrentPageDictionaryEncoded = true; + } catch (IOException e) { + throw new IOException("could not decode the dictionary for " + descriptor, e); + } + } else { + this.dictionary = null; + this.isCurrentPageDictionaryEncoded = false; + } + this.totalValueCount = pageReader.getTotalValueCount(); + if (totalValueCount == 0) { + throw new IOException("totalValueCount == 0"); + } + } + + /** + * Advances to the next value. Returns true if the value is non-null. + */ + private boolean next() throws IOException { + if (valuesRead >= endOfPageValueCount) { + if (valuesRead >= totalValueCount) { + // How do we get here? Throw end of stream exception? + return false; + } + readPage(); + } + ++valuesRead; + // TODO: Don't read for flat schemas + //repetitionLevel = repetitionLevelColumn.nextInt(); + return definitionLevelColumn.nextInt() == maxDefLevel; + } + + /** + * Reads `total` values from this columnReader into column. + */ + void readBatch(int total, ColumnVector column) throws IOException { + int rowId = 0; + ColumnVector dictionaryIds = null; + if (dictionary != null) { + // SPARK-16334: We only maintain a single dictionary per row batch, so that it can be used to + // decode all previous dictionary encoded pages if we ever encounter a non-dictionary encoded + // page. + dictionaryIds = column.reserveDictionaryIds(total); + } + while (total > 0) { + // Compute the number of values we want to read in this page. + int leftInPage = (int) (endOfPageValueCount - valuesRead); + if (leftInPage == 0) { + readPage(); + leftInPage = (int) (endOfPageValueCount - valuesRead); + } + int num = Math.min(total, leftInPage); + if (isCurrentPageDictionaryEncoded) { + // Read and decode dictionary ids. + defColumn.readIntegers( + num, dictionaryIds, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + + // Timestamp values encoded as INT64 can't be lazily decoded as we need to post process + // the values to add microseconds precision. + if (column.hasDictionary() || (rowId == 0 && + (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT32 || + (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT64 && + column.dataType() != DataTypes.TimestampType) || + descriptor.getType() == PrimitiveType.PrimitiveTypeName.FLOAT || + descriptor.getType() == PrimitiveType.PrimitiveTypeName.DOUBLE || + descriptor.getType() == PrimitiveType.PrimitiveTypeName.BINARY))) { + // Column vector supports lazy decoding of dictionary values so just set the dictionary. + // We can't do this if rowId != 0 AND the column doesn't have a dictionary (i.e. some + // non-dictionary encoded values have already been added). + column.setDictionary(dictionary); + } else { + decodeDictionaryIds(rowId, num, column, dictionaryIds); + } + } else { + if (column.hasDictionary() && rowId != 0) { + // This batch already has dictionary encoded values but this new page is not. The batch + // does not support a mix of dictionary and not so we will decode the dictionary. + decodeDictionaryIds(0, rowId, column, column.getDictionaryIds()); + } + column.setDictionary(null); + switch (descriptor.getType()) { + case BOOLEAN: + readBooleanBatch(rowId, num, column); + break; + case INT32: + readIntBatch(rowId, num, column); + break; + case INT64: + readLongBatch(rowId, num, column); + break; + case INT96: + readBinaryBatch(rowId, num, column); + break; + case FLOAT: + readFloatBatch(rowId, num, column); + break; + case DOUBLE: + readDoubleBatch(rowId, num, column); + break; + case BINARY: + readBinaryBatch(rowId, num, column); + break; + case FIXED_LEN_BYTE_ARRAY: + readFixedLenByteArrayBatch(rowId, num, column, descriptor.getTypeLength()); + break; + default: + throw new IOException("Unsupported type: " + descriptor.getType()); + } + } + + valuesRead += num; + rowId += num; + total -= num; + } + } + + /** + * Reads `num` values into column, decoding the values from `dictionaryIds` and `dictionary`. + */ + private void decodeDictionaryIds(int rowId, int num, ColumnVector column, + ColumnVector dictionaryIds) { + switch (descriptor.getType()) { + case INT32: + if (column.dataType() == DataTypes.IntegerType || + DecimalType.is32BitDecimalType(column.dataType())) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.putInt(i, dictionary.decodeToInt(dictionaryIds.getDictId(i))); + } + } + } else if (column.dataType() == DataTypes.ByteType) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.putByte(i, (byte) dictionary.decodeToInt(dictionaryIds.getDictId(i))); + } + } + } else if (column.dataType() == DataTypes.ShortType) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.putShort(i, (short) dictionary.decodeToInt(dictionaryIds.getDictId(i))); + } + } + } else { + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); + } + break; + + case INT64: + if (column.dataType() == DataTypes.LongType || + DecimalType.is64BitDecimalType(column.dataType())) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.putLong(i, dictionary.decodeToLong(dictionaryIds.getDictId(i))); + } + } + } else if (column.dataType() == DataTypes.TimestampType) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.putLong(i, + DateTimeUtils.fromMillis(dictionary.decodeToLong(dictionaryIds.getDictId(i)))); + } + } + } + else { + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); + } + break; + + case FLOAT: + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.putFloat(i, dictionary.decodeToFloat(dictionaryIds.getDictId(i))); + } + } + break; + + case DOUBLE: + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + column.putDouble(i, dictionary.decodeToDouble(dictionaryIds.getDictId(i))); + } + } + break; + case INT96: + if (column.dataType() == DataTypes.TimestampType) { + for (int i = rowId; i < rowId + num; ++i) { + // TODO: Convert dictionary of Binaries to dictionary of Longs + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putLong(i, ParquetRowConverter.binaryToSQLTimestamp(v)); + } + } + } else { + throw new UnsupportedOperationException(); + } + break; + case BINARY: + // TODO: this is incredibly inefficient as it blows up the dictionary right here. We + // need to do this better. We should probably add the dictionary data to the ColumnVector + // and reuse it across batches. This should mean adding a ByteArray would just update + // the length and offset. + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putByteArray(i, v.getBytes()); + } + } + break; + case FIXED_LEN_BYTE_ARRAY: + // DecimalType written in the legacy mode + if (DecimalType.is32BitDecimalType(column.dataType())) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putInt(i, (int) ParquetRowConverter.binaryToUnscaledLong(v)); + } + } + } else if (DecimalType.is64BitDecimalType(column.dataType())) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putLong(i, ParquetRowConverter.binaryToUnscaledLong(v)); + } + } + } else if (DecimalType.isByteArrayDecimalType(column.dataType())) { + for (int i = rowId; i < rowId + num; ++i) { + if (!column.isNullAt(i)) { + Binary v = dictionary.decodeToBinary(dictionaryIds.getDictId(i)); + column.putByteArray(i, v.getBytes()); + } + } + } else { + throw new UnsupportedOperationException(); + } + break; + + default: + throw new UnsupportedOperationException("Unsupported type: " + descriptor.getType()); + } + } + + /** + * For all the read*Batch functions, reads `num` values from this columnReader into column. It + * is guaranteed that num is smaller than the number of values left in the current page. + */ + + private void readBooleanBatch(int rowId, int num, ColumnVector column) throws IOException { + assert(column.dataType() == DataTypes.BooleanType); + defColumn.readBooleans( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } + + private void readIntBatch(int rowId, int num, ColumnVector column) throws IOException { + // This is where we implement support for the valid type conversions. + // TODO: implement remaining type conversions + if (column.dataType() == DataTypes.IntegerType || column.dataType() == DataTypes.DateType || + DecimalType.is32BitDecimalType(column.dataType())) { + defColumn.readIntegers( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } else if (column.dataType() == DataTypes.ByteType) { + defColumn.readBytes( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } else if (column.dataType() == DataTypes.ShortType) { + defColumn.readShorts( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } else { + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); + } + } + + private void readLongBatch(int rowId, int num, ColumnVector column) throws IOException { + // This is where we implement support for the valid type conversions. + if (column.dataType() == DataTypes.LongType || + DecimalType.is64BitDecimalType(column.dataType())) { + defColumn.readLongs( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } else if (column.dataType() == DataTypes.TimestampType) { + for (int i = 0; i < num; i++) { + if (defColumn.readInteger() == maxDefLevel) { + column.putLong(rowId + i, DateTimeUtils.fromMillis(dataColumn.readLong())); + } else { + column.putNull(rowId + i); + } + } + } else { + throw new UnsupportedOperationException("Unsupported conversion to: " + column.dataType()); + } + } + + private void readFloatBatch(int rowId, int num, ColumnVector column) throws IOException { + // This is where we implement support for the valid type conversions. + // TODO: support implicit cast to double? + if (column.dataType() == DataTypes.FloatType) { + defColumn.readFloats( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } else { + throw new UnsupportedOperationException("Unsupported conversion to: " + column.dataType()); + } + } + + private void readDoubleBatch(int rowId, int num, ColumnVector column) throws IOException { + // This is where we implement support for the valid type conversions. + // TODO: implement remaining type conversions + if (column.dataType() == DataTypes.DoubleType) { + defColumn.readDoubles( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } else { + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); + } + } + + private void readBinaryBatch(int rowId, int num, ColumnVector column) throws IOException { + // This is where we implement support for the valid type conversions. + // TODO: implement remaining type conversions + VectorizedValuesReader data = (VectorizedValuesReader) dataColumn; + if (column.isArray()) { + defColumn.readBinarys(num, column, rowId, maxDefLevel, data); + } else if (column.dataType() == DataTypes.TimestampType) { + for (int i = 0; i < num; i++) { + if (defColumn.readInteger() == maxDefLevel) { + column.putLong(rowId + i, + // Read 12 bytes for INT96 + ParquetRowConverter.binaryToSQLTimestamp(data.readBinary(12))); + } else { + column.putNull(rowId + i); + } + } + } else { + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); + } + } + + private void readFixedLenByteArrayBatch(int rowId, int num, + ColumnVector column, int arrayLen) throws IOException { + VectorizedValuesReader data = (VectorizedValuesReader) dataColumn; + // This is where we implement support for the valid type conversions. + // TODO: implement remaining type conversions + if (DecimalType.is32BitDecimalType(column.dataType())) { + for (int i = 0; i < num; i++) { + if (defColumn.readInteger() == maxDefLevel) { + column.putInt(rowId + i, + (int) ParquetRowConverter.binaryToUnscaledLong(data.readBinary(arrayLen))); + } else { + column.putNull(rowId + i); + } + } + } else if (DecimalType.is64BitDecimalType(column.dataType())) { + for (int i = 0; i < num; i++) { + if (defColumn.readInteger() == maxDefLevel) { + column.putLong(rowId + i, + ParquetRowConverter.binaryToUnscaledLong(data.readBinary(arrayLen))); + } else { + column.putNull(rowId + i); + } + } + } else if (DecimalType.isByteArrayDecimalType(column.dataType())) { + for (int i = 0; i < num; i++) { + if (defColumn.readInteger() == maxDefLevel) { + column.putByteArray(rowId + i, data.readBinary(arrayLen).getBytes()); + } else { + column.putNull(rowId + i); + } + } + } else { + throw new UnsupportedOperationException("Unimplemented type: " + column.dataType()); + } + } + + private void readPage() throws IOException { + DataPage page = pageReader.readPage(); + // TODO: Why is this a visitor? + page.accept(new DataPage.Visitor() { + @Override + public Void visit(DataPageV1 dataPageV1) { + try { + readPageV1(dataPageV1); + return null; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public Void visit(DataPageV2 dataPageV2) { + try { + readPageV2(dataPageV2); + return null; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }); + } + + private void initDataReader(Encoding dataEncoding, byte[] bytes, int offset) throws IOException { + this.endOfPageValueCount = valuesRead + pageValueCount; + if (dataEncoding.usesDictionary()) { + this.dataColumn = null; + if (dictionary == null) { + throw new IOException( + "could not read page in col " + descriptor + + " as the dictionary was missing for encoding " + dataEncoding); + } + @SuppressWarnings("deprecation") + Encoding plainDict = Encoding.PLAIN_DICTIONARY; // var to allow warning suppression + if (dataEncoding != plainDict && dataEncoding != Encoding.RLE_DICTIONARY) { + throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding); + } + this.dataColumn = new VectorizedRleValuesReader(); + this.isCurrentPageDictionaryEncoded = true; + } else { + if (dataEncoding != Encoding.PLAIN) { + throw new UnsupportedOperationException("Unsupported encoding: " + dataEncoding); + } + this.dataColumn = new VectorizedPlainValuesReader(); + this.isCurrentPageDictionaryEncoded = false; + } + + try { + dataColumn.initFromPage(pageValueCount, bytes, offset); + } catch (IOException e) { + throw new IOException("could not read page in col " + descriptor, e); + } + } + + private void readPageV1(DataPageV1 page) throws IOException { + this.pageValueCount = page.getValueCount(); + ValuesReader rlReader = page.getRlEncoding().getValuesReader(descriptor, REPETITION_LEVEL); + ValuesReader dlReader; + + // Initialize the decoders. + if (page.getDlEncoding() != Encoding.RLE && descriptor.getMaxDefinitionLevel() != 0) { + throw new UnsupportedOperationException("Unsupported encoding: " + page.getDlEncoding()); + } + int bitWidth = BytesUtils.getWidthFromMaxInt(descriptor.getMaxDefinitionLevel()); + this.defColumn = new VectorizedRleValuesReader(bitWidth); + dlReader = this.defColumn; + this.repetitionLevelColumn = new ValuesReaderIntIterator(rlReader); + this.definitionLevelColumn = new ValuesReaderIntIterator(dlReader); + try { + byte[] bytes = page.getBytes().toByteArray(); + rlReader.initFromPage(pageValueCount, bytes, 0); + int next = rlReader.getNextOffset(); + dlReader.initFromPage(pageValueCount, bytes, next); + next = dlReader.getNextOffset(); + initDataReader(page.getValueEncoding(), bytes, next); + } catch (IOException e) { + throw new IOException("could not read page " + page + " in col " + descriptor, e); + } + } + + private void readPageV2(DataPageV2 page) throws IOException { + this.pageValueCount = page.getValueCount(); + this.repetitionLevelColumn = createRLEIterator(descriptor.getMaxRepetitionLevel(), + page.getRepetitionLevels(), descriptor); + + int bitWidth = BytesUtils.getWidthFromMaxInt(descriptor.getMaxDefinitionLevel()); + this.defColumn = new VectorizedRleValuesReader(bitWidth); + this.definitionLevelColumn = new ValuesReaderIntIterator(this.defColumn); + this.defColumn.initFromBuffer( + this.pageValueCount, page.getDefinitionLevels().toByteArray()); + try { + initDataReader(page.getDataEncoding(), page.getData().toByteArray(), 0); + } catch (IOException e) { + throw new IOException("could not read page " + page + " in col " + descriptor, e); + } + } +} diff --git a/src/main/spark2.2/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java b/src/main/spark2.2/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java new file mode 100644 index 00000000000..98018b7f48b --- /dev/null +++ b/src/main/spark2.2/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java @@ -0,0 +1,177 @@ +/* + * 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.datasources.parquet; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +import org.apache.spark.sql.execution.vectorized.ColumnVector; +import org.apache.spark.unsafe.Platform; + +import org.apache.parquet.column.values.ValuesReader; +import org.apache.parquet.io.api.Binary; + +/** + * An implementation of the Parquet PLAIN decoder that supports the vectorized interface. + */ +public class VectorizedPlainValuesReader extends ValuesReader implements VectorizedValuesReader { + private byte[] buffer; + private int offset; + private int bitOffset; // Only used for booleans. + private ByteBuffer byteBuffer; // used to wrap the byte array buffer + + private static final boolean bigEndianPlatform = + ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); + + public VectorizedPlainValuesReader() { + } + + @Override + public void initFromPage(int valueCount, byte[] bytes, int offset) throws IOException { + this.buffer = bytes; + this.offset = offset + Platform.BYTE_ARRAY_OFFSET; + if (bigEndianPlatform) { + byteBuffer = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN); + } + } + + @Override + public void skip() { + throw new UnsupportedOperationException(); + } + + @Override + public final void readBooleans(int total, ColumnVector c, int rowId) { + // TODO: properly vectorize this + for (int i = 0; i < total; i++) { + c.putBoolean(rowId + i, readBoolean()); + } + } + + @Override + public final void readIntegers(int total, ColumnVector c, int rowId) { + c.putIntsLittleEndian(rowId, total, buffer, offset - Platform.BYTE_ARRAY_OFFSET); + offset += 4 * total; + } + + @Override + public final void readLongs(int total, ColumnVector c, int rowId) { + c.putLongsLittleEndian(rowId, total, buffer, offset - Platform.BYTE_ARRAY_OFFSET); + offset += 8 * total; + } + + @Override + public final void readFloats(int total, ColumnVector c, int rowId) { + c.putFloats(rowId, total, buffer, offset - Platform.BYTE_ARRAY_OFFSET); + offset += 4 * total; + } + + @Override + public final void readDoubles(int total, ColumnVector c, int rowId) { + c.putDoubles(rowId, total, buffer, offset - Platform.BYTE_ARRAY_OFFSET); + offset += 8 * total; + } + + @Override + public final void readBytes(int total, ColumnVector c, int rowId) { + for (int i = 0; i < total; i++) { + // Bytes are stored as a 4-byte little endian int. Just read the first byte. + // TODO: consider pushing this in ColumnVector by adding a readBytes with a stride. + c.putByte(rowId + i, Platform.getByte(buffer, offset)); + offset += 4; + } + } + + @Override + public final boolean readBoolean() { + byte b = Platform.getByte(buffer, offset); + boolean v = (b & (1 << bitOffset)) != 0; + bitOffset += 1; + if (bitOffset == 8) { + bitOffset = 0; + offset++; + } + return v; + } + + @Override + public final int readInteger() { + int v = Platform.getInt(buffer, offset); + if (bigEndianPlatform) { + v = java.lang.Integer.reverseBytes(v); + } + offset += 4; + return v; + } + + @Override + public final long readLong() { + long v = Platform.getLong(buffer, offset); + if (bigEndianPlatform) { + v = java.lang.Long.reverseBytes(v); + } + offset += 8; + return v; + } + + @Override + public final byte readByte() { + return (byte)readInteger(); + } + + @Override + public final float readFloat() { + float v; + if (!bigEndianPlatform) { + v = Platform.getFloat(buffer, offset); + } else { + v = byteBuffer.getFloat(offset - Platform.BYTE_ARRAY_OFFSET); + } + offset += 4; + return v; + } + + @Override + public final double readDouble() { + double v; + if (!bigEndianPlatform) { + v = Platform.getDouble(buffer, offset); + } else { + v = byteBuffer.getDouble(offset - Platform.BYTE_ARRAY_OFFSET); + } + offset += 8; + return v; + } + + @Override + public final void readBinary(int total, ColumnVector v, int rowId) { + for (int i = 0; i < total; i++) { + int len = readInteger(); + int start = offset; + offset += len; + v.putByteArray(rowId + i, buffer, start - Platform.BYTE_ARRAY_OFFSET, len); + } + } + + @Override + public final Binary readBinary(int len) { + Binary result = Binary.fromConstantByteArray(buffer, offset - Platform.BYTE_ARRAY_OFFSET, len); + offset += len; + return result; + } +} diff --git a/src/main/spark2.2/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedRleValuesReader.java b/src/main/spark2.2/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedRleValuesReader.java new file mode 100644 index 00000000000..62157389013 --- /dev/null +++ b/src/main/spark2.2/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedRleValuesReader.java @@ -0,0 +1,613 @@ +/* + * 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.datasources.parquet; + +import org.apache.parquet.Preconditions; +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.values.ValuesReader; +import org.apache.parquet.column.values.bitpacking.BytePacker; +import org.apache.parquet.column.values.bitpacking.Packer; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.io.api.Binary; + +import org.apache.spark.sql.execution.vectorized.ColumnVector; + +/** + * A values reader for Parquet's run-length encoded data. This is based off of the version in + * parquet-mr with these changes: + * - Supports the vectorized interface. + * - Works on byte arrays(byte[]) instead of making byte streams. + * + * This encoding is used in multiple places: + * - Definition/Repetition levels + * - Dictionary ids. + */ +public final class VectorizedRleValuesReader extends ValuesReader + implements VectorizedValuesReader { + // Current decoding mode. The encoded data contains groups of either run length encoded data + // (RLE) or bit packed data. Each group contains a header that indicates which group it is and + // the number of values in the group. + // More details here: https://github.com/Parquet/parquet-format/blob/master/Encodings.md + private enum MODE { + RLE, + PACKED + } + + // Encoded data. + private byte[] in; + private int end; + private int offset; + + // bit/byte width of decoded data and utility to batch unpack them. + private int bitWidth; + private int bytesWidth; + private BytePacker packer; + + // Current decoding mode and values + private MODE mode; + private int currentCount; + private int currentValue; + + // Buffer of decoded values if the values are PACKED. + private int[] currentBuffer = new int[16]; + private int currentBufferIdx = 0; + + // If true, the bit width is fixed. This decoder is used in different places and this also + // controls if we need to read the bitwidth from the beginning of the data stream. + private final boolean fixedWidth; + + public VectorizedRleValuesReader() { + fixedWidth = false; + } + + public VectorizedRleValuesReader(int bitWidth) { + fixedWidth = true; + init(bitWidth); + } + + @Override + public void initFromPage(int valueCount, byte[] page, int start) { + this.offset = start; + this.in = page; + if (fixedWidth) { + if (bitWidth != 0) { + int length = readIntLittleEndian(); + this.end = this.offset + length; + } + } else { + this.end = page.length; + if (this.end != this.offset) init(page[this.offset++] & 255); + } + if (bitWidth == 0) { + // 0 bit width, treat this as an RLE run of valueCount number of 0's. + this.mode = MODE.RLE; + this.currentCount = valueCount; + this.currentValue = 0; + } else { + this.currentCount = 0; + } + } + + // Initialize the reader from a buffer. This is used for the V2 page encoding where the + // definition are in its own buffer. + public void initFromBuffer(int valueCount, byte[] data) { + this.offset = 0; + this.in = data; + this.end = data.length; + if (bitWidth == 0) { + // 0 bit width, treat this as an RLE run of valueCount number of 0's. + this.mode = MODE.RLE; + this.currentCount = valueCount; + this.currentValue = 0; + } else { + this.currentCount = 0; + } + } + + /** + * Initializes the internal state for decoding ints of `bitWidth`. + */ + private void init(int bitWidth) { + Preconditions.checkArgument(bitWidth >= 0 && bitWidth <= 32, "bitWidth must be >= 0 and <= 32"); + this.bitWidth = bitWidth; + this.bytesWidth = BytesUtils.paddedByteCountFromBits(bitWidth); + this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bitWidth); + } + + @Override + public int getNextOffset() { + return this.end; + } + + @Override + public boolean readBoolean() { + return this.readInteger() != 0; + } + + @Override + public void skip() { + this.readInteger(); + } + + @Override + public int readValueDictionaryId() { + return readInteger(); + } + + @Override + public int readInteger() { + if (this.currentCount == 0) { this.readNextGroup(); } + + this.currentCount--; + switch (mode) { + case RLE: + return this.currentValue; + case PACKED: + return this.currentBuffer[currentBufferIdx++]; + } + throw new RuntimeException("Unreachable"); + } + + /** + * Reads `total` ints into `c` filling them in starting at `c[rowId]`. This reader + * reads the definition levels and then will read from `data` for the non-null values. + * If the value is null, c will be populated with `nullValue`. Note that `nullValue` is only + * necessary for readIntegers because we also use it to decode dictionaryIds and want to make + * sure it always has a value in range. + * + * This is a batched version of this logic: + * if (this.readInt() == level) { + * c[rowId] = data.readInteger(); + * } else { + * c[rowId] = null; + * } + */ + public void readIntegers(int total, ColumnVector c, int rowId, int level, + VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readIntegers(n, c, rowId); + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + c.putInt(rowId + i, data.readInteger()); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + // TODO: can this code duplication be removed without a perf penalty? + public void readBooleans(int total, ColumnVector c, + int rowId, int level, VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readBooleans(n, c, rowId); + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + c.putBoolean(rowId + i, data.readBoolean()); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + public void readBytes(int total, ColumnVector c, + int rowId, int level, VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readBytes(n, c, rowId); + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + c.putByte(rowId + i, data.readByte()); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + public void readShorts(int total, ColumnVector c, + int rowId, int level, VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + for (int i = 0; i < n; i++) { + c.putShort(rowId + i, (short)data.readInteger()); + } + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + c.putShort(rowId + i, (short)data.readInteger()); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + public void readLongs(int total, ColumnVector c, int rowId, int level, + VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readLongs(n, c, rowId); + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + c.putLong(rowId + i, data.readLong()); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + public void readFloats(int total, ColumnVector c, int rowId, int level, + VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readFloats(n, c, rowId); + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + c.putFloat(rowId + i, data.readFloat()); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + public void readDoubles(int total, ColumnVector c, int rowId, int level, + VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readDoubles(n, c, rowId); + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + c.putDouble(rowId + i, data.readDouble()); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + public void readBinarys(int total, ColumnVector c, int rowId, int level, + VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readBinary(n, c, rowId); + } else { + c.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + data.readBinary(1, c, rowId + i); + } else { + c.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + /** + * Decoding for dictionary ids. The IDs are populated into `values` and the nullability is + * populated into `nulls`. + */ + public void readIntegers(int total, ColumnVector values, ColumnVector nulls, int rowId, int level, + VectorizedValuesReader data) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + if (currentValue == level) { + data.readIntegers(n, values, rowId); + } else { + nulls.putNulls(rowId, n); + } + break; + case PACKED: + for (int i = 0; i < n; ++i) { + if (currentBuffer[currentBufferIdx++] == level) { + values.putInt(rowId + i, data.readInteger()); + } else { + nulls.putNull(rowId + i); + } + } + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + + // The RLE reader implements the vectorized decoding interface when used to decode dictionary + // IDs. This is different than the above APIs that decodes definitions levels along with values. + // Since this is only used to decode dictionary IDs, only decoding integers is supported. + @Override + public void readIntegers(int total, ColumnVector c, int rowId) { + int left = total; + while (left > 0) { + if (this.currentCount == 0) this.readNextGroup(); + int n = Math.min(left, this.currentCount); + switch (mode) { + case RLE: + c.putInts(rowId, n, currentValue); + break; + case PACKED: + c.putInts(rowId, n, currentBuffer, currentBufferIdx); + currentBufferIdx += n; + break; + } + rowId += n; + left -= n; + currentCount -= n; + } + } + + @Override + public byte readByte() { + throw new UnsupportedOperationException("only readInts is valid."); + } + + @Override + public void readBytes(int total, ColumnVector c, int rowId) { + throw new UnsupportedOperationException("only readInts is valid."); + } + + @Override + public void readLongs(int total, ColumnVector c, int rowId) { + throw new UnsupportedOperationException("only readInts is valid."); + } + + @Override + public void readBinary(int total, ColumnVector c, int rowId) { + throw new UnsupportedOperationException("only readInts is valid."); + } + + @Override + public void readBooleans(int total, ColumnVector c, int rowId) { + throw new UnsupportedOperationException("only readInts is valid."); + } + + @Override + public void readFloats(int total, ColumnVector c, int rowId) { + throw new UnsupportedOperationException("only readInts is valid."); + } + + @Override + public void readDoubles(int total, ColumnVector c, int rowId) { + throw new UnsupportedOperationException("only readInts is valid."); + } + + @Override + public Binary readBinary(int len) { + throw new UnsupportedOperationException("only readInts is valid."); + } + + /** + * Reads the next varint encoded int. + */ + private int readUnsignedVarInt() { + int value = 0; + int shift = 0; + int b; + do { + b = in[offset++] & 255; + value |= (b & 0x7F) << shift; + shift += 7; + } while ((b & 0x80) != 0); + return value; + } + + /** + * Reads the next 4 byte little endian int. + */ + private int readIntLittleEndian() { + int ch4 = in[offset] & 255; + int ch3 = in[offset + 1] & 255; + int ch2 = in[offset + 2] & 255; + int ch1 = in[offset + 3] & 255; + offset += 4; + return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0)); + } + + /** + * Reads the next byteWidth little endian int. + */ + private int readIntLittleEndianPaddedOnBitWidth() { + switch (bytesWidth) { + case 0: + return 0; + case 1: + return in[offset++] & 255; + case 2: { + int ch2 = in[offset] & 255; + int ch1 = in[offset + 1] & 255; + offset += 2; + return (ch1 << 8) + ch2; + } + case 3: { + int ch3 = in[offset] & 255; + int ch2 = in[offset + 1] & 255; + int ch1 = in[offset + 2] & 255; + offset += 3; + return (ch1 << 16) + (ch2 << 8) + (ch3 << 0); + } + case 4: { + return readIntLittleEndian(); + } + } + throw new RuntimeException("Unreachable"); + } + + private int ceil8(int value) { + return (value + 7) / 8; + } + + /** + * Reads the next group. + */ + private void readNextGroup() { + int header = readUnsignedVarInt(); + this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED; + switch (mode) { + case RLE: + this.currentCount = header >>> 1; + this.currentValue = readIntLittleEndianPaddedOnBitWidth(); + return; + case PACKED: + int numGroups = header >>> 1; + this.currentCount = numGroups * 8; + int bytesToRead = ceil8(this.currentCount * this.bitWidth); + + if (this.currentBuffer.length < this.currentCount) { + this.currentBuffer = new int[this.currentCount]; + } + currentBufferIdx = 0; + int valueIndex = 0; + for (int byteIndex = offset; valueIndex < this.currentCount; byteIndex += this.bitWidth) { + this.packer.unpack8Values(in, byteIndex, this.currentBuffer, valueIndex); + valueIndex += 8; + } + offset += bytesToRead; + return; + default: + throw new ParquetDecodingException("not a valid mode " + this.mode); + } + } +} diff --git a/src/main/spark2.2/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java b/src/main/spark2.2/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java index 8b7b0e655b3..f09d10d896b 100644 --- a/src/main/spark2.2/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java +++ b/src/main/spark2.2/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java @@ -86,6 +86,11 @@ public void close() { } } + // For DataSourceScanExec + public boolean isFiltered(int rowId) { + return filteredRows[rowId]; + } + /** * Adapter class to interop with existing components that expect internal row. A lot of * performance is lost with this translation. @@ -145,7 +150,7 @@ public InternalRow copy() { } else if (dt instanceof LongType) { row.setLong(i, getLong(i)); } else if (dt instanceof FloatType) { - row.setFloat(i, getFloat(i)); + row.setFloat(i, getFloat(i)); } else if (dt instanceof DoubleType) { row.setDouble(i, getDouble(i)); } else if (dt instanceof StringType) { @@ -169,7 +174,7 @@ public InternalRow copy() { @Override public boolean anyNull() { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("any null"); } @Override @@ -236,12 +241,12 @@ public ArrayData getArray(int ordinal) { @Override public MapData getMap(int ordinal) { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("get map"); } @Override public Object get(int ordinal, DataType dataType) { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("use specific get"); } @Override @@ -366,7 +371,7 @@ public Row next() { @Override public void remove() { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("remove"); } }; } @@ -464,6 +469,25 @@ public void markFiltered(int rowId) { ++numRowsFiltered; } + /** + * Marks this row not filtered out. This means a subsequent iteration over the rows + * in this batch will include this row. + * For IndexedVectorizedOapRecordReader. + */ + public void markValid(int rowId) { + assert(filteredRows[rowId]); + filteredRows[rowId] = false; + --numRowsFiltered; + } + + /** + * For IndexedVectorizedOapRecordReader. + */ + public void markAllFiltered() { + Arrays.fill(filteredRows, true); + numRowsFiltered = numRows; + } + /** * Marks a given column as non-nullable. Any row that has a NULL value for the corresponding * attribute is filtered out. diff --git a/src/main/spark2.2/scala/org/apache/spark/SparkEnv.scala b/src/main/spark2.2/scala/org/apache/spark/SparkEnv.scala index f4a59f069a5..f62be8bbe67 100644 --- a/src/main/spark2.2/scala/org/apache/spark/SparkEnv.scala +++ b/src/main/spark2.2/scala/org/apache/spark/SparkEnv.scala @@ -40,6 +40,7 @@ import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinato import org.apache.spark.security.CryptoStreamUtils import org.apache.spark.serializer.{JavaSerializer, Serializer, SerializerManager} import org.apache.spark.shuffle.ShuffleManager +import org.apache.spark.sql.oap.OapRuntime import org.apache.spark.storage._ import org.apache.spark.util.{RpcUtils, Utils} @@ -84,6 +85,7 @@ class SparkEnv ( if (!isStopped) { isStopped = true pythonWorkers.values.foreach(_.stop()) + OapRuntime.stop() mapOutputTracker.stop() shuffleManager.stop() broadcastManager.stop() diff --git a/src/main/spark2.2/scala/org/apache/spark/scheduler/DAGScheduler.scala b/src/main/spark2.2/scala/org/apache/spark/scheduler/DAGScheduler.scala index 35f6b365eca..e8f05b7245e 100644 --- a/src/main/spark2.2/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/src/main/spark2.2/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -284,6 +284,30 @@ class DAGScheduler( cacheLocs(rdd.id) } + private [scheduler] + def getOapCacheLocs(rdd: RDD[_], partition: Int): Seq[TaskLocation] = this.synchronized { + val locations = rdd.preferredLocations(rdd.partitions(partition)) + + // here we check if the splits was cached, if the splits is cached, there will be hosts with + // format "OAP_HOST_host_OAP_EXECUTOR_exec", so we check the host list to filter out + // the cached hosts, so that the tasks' locality level will be PROCESS_LOCAL + if (locations.nonEmpty) { + // TODO use constant value for prefixes, these prefixes should be the same with that in + // [[org.apache.spark.sql.execution.datasources.oap.FiberSensor]] + val cacheLocs = locations.filter(_.startsWith("OAP_HOST_")) + val oapPrefs = cacheLocs.map { cacheLoc => + val host = cacheLoc.split("_OAP_EXECUTOR_")(0).stripPrefix("OAP_HOST_") + val execId = cacheLoc.split("_OAP_EXECUTOR_")(1) + (host, execId) + } + if (oapPrefs.nonEmpty) { + logDebug(s"got oap prefer location value oapPrefs is ${oapPrefs}") + return oapPrefs.map(loc => TaskLocation(loc._1, loc._2)) + } + } + Seq.empty + } + private def clearCacheLocs(): Unit = cacheLocs.synchronized { cacheLocs.clear() } @@ -1368,9 +1392,8 @@ class DAGScheduler( * modify the scheduler's internal state. Use executorLost() to post a loss event from outside. * * We will also assume that we've lost all shuffle blocks associated with the executor if the - * executor serves its own blocks (i.e., we're not using external shuffle), the entire slave - * is lost (likely including the shuffle service), or a FetchFailed occurred, in which case we - * presume all shuffle data related to this executor to be lost. + * executor serves its own blocks (i.e., we're not using external shuffle) OR a FetchFailed + * occurred, in which case we presume all shuffle data related to this executor to be lost. * * Optionally the epoch during which the failure was caught can be passed to avoid allowing * stray fetch failures from possibly retriggering the detection of a node as lost. @@ -1608,8 +1631,9 @@ class DAGScheduler( } // If the partition is cached, return the cache locations val cached = getCacheLocs(rdd)(partition) - if (cached.nonEmpty) { - return cached + val oapCached = getOapCacheLocs(rdd, partition) + if (cached.nonEmpty || oapCached.nonEmpty) { + return cached ++ oapCached } // If the RDD has some placement preferences (as is the case for input RDDs), get those val rddPrefs = rdd.preferredLocations(rdd.partitions(partition)).toList diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala b/src/main/spark2.2/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala new file mode 100644 index 00000000000..fc02dd3a5af --- /dev/null +++ b/src/main/spark2.2/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala @@ -0,0 +1,144 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.vectorized.{ColumnarBatch, ColumnVector} +import org.apache.spark.sql.types.DataType + + +/** + * Helper trait for abstracting scan functionality using + * [[org.apache.spark.sql.execution.vectorized.ColumnarBatch]]es. + */ +private[sql] trait ColumnarBatchScan extends CodegenSupport { + + val inMemoryTableScan: InMemoryTableScanExec = null + + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) + + /** + * Generate [[ColumnVector]] expressions for our parent to consume as rows. + * This is called once per [[ColumnarBatch]]. + */ + private def genCodeColumnVector( + ctx: CodegenContext, + columnVar: String, + ordinal: String, + dataType: DataType, + nullable: Boolean): ExprCode = { + val javaType = ctx.javaType(dataType) + val value = ctx.getValue(columnVar, dataType, ordinal) + val isNullVar = if (nullable) { ctx.freshName("isNull") } else { "false" } + val valueVar = ctx.freshName("value") + val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" + val code = s"${ctx.registerComment(str)}\n" + (if (nullable) { + s""" + boolean $isNullVar = $columnVar.isNullAt($ordinal); + $javaType $valueVar = $isNullVar ? ${ctx.defaultValue(dataType)} : ($value); + """ + } else { + s"$javaType $valueVar = $value;" + }).trim + ExprCode(code, isNullVar, valueVar) + } + + /** + * Produce code to process the input iterator as [[ColumnarBatch]]es. + * This produces an [[UnsafeRow]] for each row in each batch. + */ + // TODO: return ColumnarBatch.Rows instead + override protected def doProduce(ctx: CodegenContext): String = { + val input = ctx.freshName("input") + // PhysicalRDD always just has one input + ctx.addMutableState("scala.collection.Iterator", input, s"$input = inputs[0];") + + // metrics + val numOutputRows = metricTerm(ctx, "numOutputRows") + val scanTimeMetric = metricTerm(ctx, "scanTime") + val scanTimeTotalNs = ctx.freshName("scanTime") + ctx.addMutableState("long", scanTimeTotalNs, s"$scanTimeTotalNs = 0;") + + val columnarBatchClz = "org.apache.spark.sql.execution.vectorized.ColumnarBatch" + val batch = ctx.freshName("batch") + ctx.addMutableState(columnarBatchClz, batch, s"$batch = null;") + + val columnVectorClz = "org.apache.spark.sql.execution.vectorized.ColumnVector" + val idx = ctx.freshName("batchIdx") + ctx.addMutableState("int", idx, s"$idx = 0;") + val colVars = output.indices.map(i => ctx.freshName("colInstance" + i)) + val columnAssigns = colVars.zipWithIndex.map { case (name, i) => + ctx.addMutableState(columnVectorClz, name, s"$name = null;") + s"$name = $batch.column($i);" + } + + val nextBatch = ctx.freshName("nextBatch") + ctx.addNewFunction(nextBatch, + s""" + |private void $nextBatch() throws java.io.IOException { + | long getBatchStart = System.nanoTime(); + | if ($input.hasNext()) { + | $batch = ($columnarBatchClz)$input.next(); + | $numOutputRows.add($batch.numRows()); + | $idx = 0; + | ${columnAssigns.mkString("", "\n", "\n")} + | } + | $scanTimeTotalNs += System.nanoTime() - getBatchStart; + |}""".stripMargin) + + ctx.currentVars = null + val rowidx = ctx.freshName("rowIdx") + val columnsBatchInput = (output zip colVars).map { case (attr, colVar) => + genCodeColumnVector(ctx, colVar, rowidx, attr.dataType, attr.nullable) + } + val localIdx = ctx.freshName("localIdx") + val localEnd = ctx.freshName("localEnd") + val numRows = ctx.freshName("numRows") + val shouldStop = if (isShouldStopRequired) { + s"if (shouldStop()) { $idx = $rowidx + 1; return; }" + } else { + "// shouldStop check is eliminated" + } + s""" + |if ($batch == null) { + | $nextBatch(); + |} + |while ($batch != null) { + | int $numRows = $batch.numRows(); + | int $localEnd = $numRows - $idx; + | for (int $localIdx = 0; $localIdx < $localEnd; $localIdx++) { + | int $rowidx = $idx + $localIdx; + | if ($batch.isFiltered($rowidx)) continue; + | ${consume(ctx, columnsBatchInput).trim} + | $shouldStop + | } + | $idx = $numRows; + | $batch = null; + | $nextBatch(); + |} + |$scanTimeMetric.add($scanTimeTotalNs / (1000 * 1000)); + |$scanTimeTotalNs = 0; + """.stripMargin + } + +} diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/src/main/spark2.2/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 74fc23a52a1..07c9f9d93a1 100644 --- a/src/main/spark2.2/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/src/main/spark2.2/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -31,9 +31,11 @@ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.oap.{OapMetricsManager, OapFileFormat} import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.oap.OapRuntime import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -281,6 +283,13 @@ case class FileSourceScanExec( } private lazy val inputRDD: RDD[InternalRow] = { + // init accumulator before buildReader + relation.fileFormat match { + case format: OapFileFormat => + format.initMetrics(metrics) + case _ => Unit + } + val readFile: (PartitionedFile) => Iterator[InternalRow] = relation.fileFormat.buildReaderWithPartitionValues( sparkSession = relation.sparkSession, @@ -307,7 +316,8 @@ case class FileSourceScanExec( Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of files"), "metadataTime" -> SQLMetrics.createMetric(sparkContext, "metadata time (ms)"), - "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) + "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) ++ + OapMetricsManager.metrics(sparkContext) protected override def doExecute(): RDD[InternalRow] = { if (supportsBatch) { @@ -437,9 +447,10 @@ case class FileSourceScanExec( partition.values, file.getPath.toUri.toString, offset, size, hosts) } } else { - val hosts = getBlockHosts(blockLocations, 0, file.getLen) + val cachedHosts = OapRuntime.getOrCreate.fiberSensor.getHosts(file.getPath.toString) + val hosts = cachedHosts.toBuffer ++ getBlockHosts(blockLocations, 0, file.getLen) Seq(PartitionedFile( - partition.values, file.getPath.toUri.toString, 0, file.getLen, hosts)) + partition.values, file.getPath.toUri.toString, 0, file.getLen, hosts.toArray)) } } }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/src/main/spark2.2/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index c2c52894860..de8060b65a0 100644 --- a/src/main/spark2.2/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/src/main/spark2.2/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, _} +import org.apache.spark.sql.execution.datasources.oap.index._ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution} import org.apache.spark.sql.types.StructType @@ -1483,4 +1484,85 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { query: LogicalPlan): LogicalPlan = { RepartitionByExpression(expressions, query, conf.numShufflePartitions) } + + + /** + * Create an index. Create a [[CreateIndexCommand]] command. + * + * {{{ + * CREATE OINDEX [IF NOT EXISTS] indexName ON tableName (col1 [ASC | DESC], col2, ...) + * [USING (BTREE | BITMAP)] [PARTITION (partcol1=val1, partcol2=val2 ...)] + * }}} + */ + override def visitOapCreateIndex(ctx: OapCreateIndexContext): LogicalPlan = + withOrigin(ctx) { + CreateIndexCommand( + ctx.IDENTIFIER.getText, + visitTableIdentifier(ctx.tableIdentifier()), + visitIndexCols(ctx.indexCols), + ctx.EXISTS != null, visitIndexType(ctx.indexType), + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) + } + + /** + * Drop an index. Create a [[DropIndexCommand]] command. + * + * {{{ + * DROP OINDEX [IF EXISTS] indexName on tableName [PARTITION (partcol1=val1, partcol2=val2 ...)] + * }}} + */ + override def visitOapDropIndex(ctx: OapDropIndexContext): LogicalPlan = withOrigin(ctx) { + DropIndexCommand( + ctx.IDENTIFIER.getText, + visitTableIdentifier(ctx.tableIdentifier), + ctx.EXISTS != null, + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) + } + + override def visitIndexCols(ctx: IndexColsContext): Array[IndexColumn] = withOrigin(ctx) { + ctx.indexCol.toArray(new Array[IndexColContext](ctx.indexCol.size)).map(visitIndexCol) + } + + override def visitIndexCol(ctx: IndexColContext): IndexColumn = withOrigin(ctx) { + IndexColumn(ctx.identifier.getText, ctx.DESC == null) + } + + override def visitIndexType(ctx: IndexTypeContext): OapIndexType = if (ctx == null) { + BTreeIndexType + } else { + withOrigin(ctx) { + if (ctx.BTREE != null) { + BTreeIndexType + } else { + BitMapIndexType + } + } + } + + override def visitOapRefreshIndices(ctx: OapRefreshIndicesContext): LogicalPlan = + withOrigin(ctx) { + RefreshIndexCommand( + visitTableIdentifier(ctx.tableIdentifier), + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) + } + + override def visitOapShowIndex(ctx: OapShowIndexContext): LogicalPlan = withOrigin(ctx) { + val tableName = visitTableIdentifier(ctx.tableIdentifier) + OapShowIndexCommand(tableName, tableName.identifier) + } + + override def visitOapCheckIndex(ctx: OapCheckIndexContext): LogicalPlan = + withOrigin(ctx) { + val tableIdentifier = visitTableIdentifier(ctx.tableIdentifier) + OapCheckIndexCommand( + tableIdentifier, + tableIdentifier.identifier, + Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec)) + } + + override def visitOapDisableIndex(ctx: OapDisableIndexContext): LogicalPlan = + OapDisableIndexCommand(ctx.IDENTIFIER.getText) + + override def visitOapEnableIndex(ctx: OapEnableIndexContext): LogicalPlan = + OapEnableIndexCommand(ctx.IDENTIFIER.getText) } diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 0915bd3ba25..5f048bab2bf 100644 --- a/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider import org.apache.spark.sql.execution.datasources.json.JsonFileFormat +import org.apache.spark.sql.execution.datasources.oap.OapFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.sources._ @@ -488,6 +489,7 @@ object DataSource extends Logging { val csv = classOf[CSVFileFormat].getCanonicalName val libsvm = "org.apache.spark.ml.source.libsvm.LibSVMFileFormat" val orc = "org.apache.spark.sql.hive.orc.OrcFileFormat" + val oap = classOf[OapFileFormat].getCanonicalName Map( "org.apache.spark.sql.jdbc" -> jdbc, @@ -506,7 +508,8 @@ object DataSource extends Logging { "org.apache.spark.sql.hive.orc" -> orc, "org.apache.spark.ml.source.libsvm.DefaultSource" -> libsvm, "org.apache.spark.ml.source.libsvm" -> libsvm, - "com.databricks.spark.csv" -> csv + "com.databricks.spark.csv" -> csv, + "spn" -> oap ) } diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index ded9303de55..87651d302f7 100644 --- a/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -515,6 +515,9 @@ object DataSourceStrategy { case expressions.Contains(a: Attribute, Literal(v: UTF8String, StringType)) => Some(sources.StringContains(a.name, v.toString)) + case expressions.Like(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringLike(a.name, v.toString)) + case _ => None } } diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 2c31d2a84c2..cd4cf7cad6d 100644 --- a/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -81,7 +81,10 @@ object FileFormatWriter extends Logging { } /** The result of a successful write task. */ - private case class WriteTaskResult(commitMsg: TaskCommitMessage, updatedPartitions: Set[String]) + private case class WriteTaskResult( + commitMsg: TaskCommitMessage, + updatedPartitions: Set[String], + writeResults: Seq[WriteResult]) /** * Basic work flow of this command is: @@ -104,7 +107,7 @@ object FileFormatWriter extends Logging { partitionColumnNames: Seq[String], bucketSpec: Option[BucketSpec], refreshFunction: (Seq[TablePartitionSpec]) => Unit, - options: Map[String, String]): Unit = { + options: Map[String, String]): Seq[WriteResult] = { val job = Job.getInstance(hadoopConf) job.setOutputKeyClass(classOf[Void]) @@ -205,10 +208,13 @@ object FileFormatWriter extends Logging { val commitMsgs = ret.map(_.commitMsg) val updatedPartitions = ret.flatMap(_.updatedPartitions) .distinct.map(PartitioningUtils.parsePathFragment) + val writeResults = ret.flatMap(_.writeResults) committer.commitJob(job, commitMsgs) + outputWriterFactory.commitJob(writeResults) logInfo(s"Job ${job.getJobID} committed.") refreshFunction(updatedPartitions) + writeResults } catch { case cause: Throwable => logError(s"Aborting job ${job.getJobID}.", cause) committer.abortJob(job) @@ -255,9 +261,9 @@ object FileFormatWriter extends Logging { try { Utils.tryWithSafeFinallyAndFailureCallbacks(block = { // Execute the task to write rows out and commit the task. - val outputPartitions = writeTask.execute(iterator) - writeTask.releaseResources() - WriteTaskResult(committer.commitTask(taskAttemptContext), outputPartitions) + var (outputPartitions, writeResults) = writeTask.execute(iterator) + writeResults = writeResults ++ writeTask.releaseResources() + WriteTaskResult(committer.commitTask(taskAttemptContext), outputPartitions, writeResults) })(catchBlock = { // If there is an error, release resource and then abort the task try { @@ -283,8 +289,8 @@ object FileFormatWriter extends Logging { * Writes data out to files, and then returns the list of partition strings written out. * The list of partitions is sent back to the driver and used to update the catalog. */ - def execute(iterator: Iterator[InternalRow]): Set[String] - def releaseResources(): Unit + def execute(iterator: Iterator[InternalRow]): (Set[String], Seq[WriteResult]) + def releaseResources(): Seq[WriteResult] } /** Writes data to a single directory (used for non-dynamic-partition writes). */ @@ -308,9 +314,10 @@ object FileFormatWriter extends Logging { context = taskAttemptContext) } - override def execute(iter: Iterator[InternalRow]): Set[String] = { + override def execute(iter: Iterator[InternalRow]): (Set[String], Seq[WriteResult]) = { var fileCounter = 0 var recordsInFile: Long = 0L + var writeResults: Seq[WriteResult] = Nil newOutputWriter(fileCounter) while (iter.hasNext) { if (description.maxRecordsPerFile > 0 && recordsInFile >= description.maxRecordsPerFile) { @@ -319,7 +326,7 @@ object FileFormatWriter extends Logging { s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER") recordsInFile = 0 - releaseResources() + writeResults = writeResults ++ releaseResources() newOutputWriter(fileCounter) } @@ -327,18 +334,21 @@ object FileFormatWriter extends Logging { currentWriter.write(internalRow) recordsInFile += 1 } - releaseResources() - Set.empty + writeResults = writeResults ++ releaseResources() + (Set.empty, writeResults) } - override def releaseResources(): Unit = { + override def releaseResources(): Seq[WriteResult] = { + var writeResults: Seq[WriteResult] = Nil if (currentWriter != null) { try { currentWriter.close() + writeResults = writeResults :+ currentWriter.writeStatus() } finally { currentWriter = null } } + writeResults } } @@ -403,11 +413,11 @@ object FileFormatWriter extends Logging { val ext = f"$bucketId.c$fileCounter%03d" + desc.outputWriterFactory.getFileExtension(taskAttemptContext) - val customPath = partDir match { + val (customPath, partitionString) = partDir match { case Some(dir) => - desc.customPartitionLocations.get(PartitioningUtils.parsePathFragment(dir)) + (desc.customPartitionLocations.get(PartitioningUtils.parsePathFragment(dir)), dir) case _ => - None + (None, "") } val path = if (customPath.isDefined) { committer.newTaskTempFileAbsPath(taskAttemptContext, customPath.get, ext) @@ -419,9 +429,10 @@ object FileFormatWriter extends Logging { path = path, dataSchema = desc.dataColumns.toStructType, context = taskAttemptContext) + currentWriter.setPartitionString(partitionString) } - override def execute(iter: Iterator[InternalRow]): Set[String] = { + override def execute(iter: Iterator[InternalRow]): (Set[String], Seq[WriteResult]) = { val getPartitionColsAndBucketId = UnsafeProjection.create( desc.partitionColumns ++ desc.bucketIdExpression, desc.allColumns) @@ -433,6 +444,7 @@ object FileFormatWriter extends Logging { val getOutputRow = UnsafeProjection.create(desc.dataColumns, desc.allColumns) // If anything below fails, we should abort the task. + var writeResults: Seq[WriteResult] = Nil var recordsInFile: Long = 0L var fileCounter = 0 var currentPartColsAndBucketId: UnsafeRow = null @@ -447,7 +459,7 @@ object FileFormatWriter extends Logging { recordsInFile = 0 fileCounter = 0 - releaseResources() + writeResults = writeResults ++ releaseResources() newOutputWriter(currentPartColsAndBucketId, getPartPath, fileCounter, updatedPartitions) } else if (desc.maxRecordsPerFile > 0 && recordsInFile >= desc.maxRecordsPerFile) { @@ -458,25 +470,28 @@ object FileFormatWriter extends Logging { assert(fileCounter < MAX_FILE_COUNTER, s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER") - releaseResources() + writeResults = writeResults ++ releaseResources() newOutputWriter(currentPartColsAndBucketId, getPartPath, fileCounter, updatedPartitions) } currentWriter.write(getOutputRow(row)) recordsInFile += 1 } - releaseResources() - updatedPartitions.toSet + writeResults = writeResults ++ releaseResources() + (updatedPartitions.toSet, writeResults) } - override def releaseResources(): Unit = { + override def releaseResources(): Seq[WriteResult] = { + var writeResults: Seq[WriteResult] = Nil if (currentWriter != null) { try { currentWriter.close() + writeResults = writeResults :+ currentWriter.writeStatus() } finally { currentWriter = null } } + writeResults } } } diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index 17f7e0e601c..0720c473801 100644 --- a/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -23,8 +23,11 @@ import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.FileSourceScanExec -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} +import org.apache.spark.sql.execution.datasources.oap.OapFileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.oap.OapConf /** * A strategy for planning scans over collections of files that might be partitioned or bucketed @@ -52,7 +55,7 @@ import org.apache.spark.sql.execution.SparkPlan object FileSourceStrategy extends Strategy with Logging { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case PhysicalOperation(projects, filters, - l @ LogicalRelation(fsRelation: HadoopFsRelation, _, table)) => + l @ LogicalRelation(_fsRelation: HadoopFsRelation, _, table)) => // Filters on this relation fall into four categories based on where we can use them to avoid // reading unneeded data: // - partition keys only - used to prune directories to read @@ -73,12 +76,61 @@ object FileSourceStrategy extends Strategy with Logging { val partitionColumns = l.resolve( - fsRelation.partitionSchema, fsRelation.sparkSession.sessionState.analyzer.resolver) + _fsRelation.partitionSchema, _fsRelation.sparkSession.sessionState.analyzer.resolver) val partitionSet = AttributeSet(partitionColumns) val partitionKeyFilters = ExpressionSet(normalizedFilters.filter(_.references.subsetOf(partitionSet))) logInfo(s"Pruning directories with: ${partitionKeyFilters.mkString(",")}") + val selectedPartitions = _fsRelation.location.listFiles(partitionKeyFilters.toSeq, Nil) + + val fsRelation: HadoopFsRelation = _fsRelation.fileFormat match { + // TODO a better rule to check if we need to substitute the ParquetFileFormat + // as OapFileFormat + // add spark.sql.oap.parquet.enable config + // if config true turn to OapFileFormat + // else turn to ParquetFileFormat + case a: ParquetFileFormat + if _fsRelation.sparkSession.conf.get(OapConf.OAP_PARQUET_ENABLED) => + val oapFileFormat = new OapFileFormat + oapFileFormat + .init(_fsRelation.sparkSession, + _fsRelation.options, + selectedPartitions.flatMap(p => p.files)) + + if (oapFileFormat.hasAvailableIndex(normalizedFilters)) { + logInfo("hasAvailableIndex = true, will replace with OapFileFormat.") + val parquetOptions: Map[String, String] = + Map(SQLConf.PARQUET_BINARY_AS_STRING.key -> + _fsRelation.sparkSession.sessionState.conf.isParquetBinaryAsString.toString, + SQLConf.PARQUET_INT96_AS_TIMESTAMP.key -> + _fsRelation.sparkSession.sessionState.conf.isParquetINT96AsTimestamp.toString, + SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> + _fsRelation.sparkSession.sessionState.conf.writeLegacyParquetFormat.toString, + SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key -> + _fsRelation.sparkSession.sessionState.conf + .isParquetINT64AsTimestampMillis.toString) ++ + _fsRelation.options + + _fsRelation.copy(fileFormat = oapFileFormat, + options = parquetOptions)(_fsRelation.sparkSession) + + } else { + logInfo("hasAvailableIndex = false, will retain ParquetFileFormat.") + _fsRelation + } + + case _: OapFileFormat => + _fsRelation.fileFormat.asInstanceOf[OapFileFormat].init( + _fsRelation.sparkSession, + _fsRelation.options, + selectedPartitions.flatMap(p => p.files)) + _fsRelation + + case _: FileFormat => + _fsRelation + } + val dataColumns = l.resolve(fsRelation.dataSchema, fsRelation.sparkSession.sessionState.analyzer.resolver) diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala b/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala index 868e5371426..afdc4927efb 100644 --- a/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala +++ b/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala @@ -47,6 +47,13 @@ abstract class OutputWriterFactory extends Serializable { path: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter + + /** + * This API called from the driver when all of the write task finished, and give the data + * source extensions to pass back the data writing task status, etc. writing the global + * meta information. + */ + def commitJob(taskResults: Array[WriteResult]): Unit = { } } @@ -68,4 +75,14 @@ abstract class OutputWriter { * the task output is committed. */ def close(): Unit + + /** + * This is to collect data for meta when OAP writing or index writing. + * Can be replaced with datasource v2 api from Spark 2.3 + */ + def writeStatus(): WriteResult = { } + + protected[sql] def setPartitionString(ps: String): Unit = { + // do nothing + } } diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/oap/OapOutputWriter.scala b/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/oap/OapOutputWriter.scala new file mode 100644 index 00000000000..ff768299b37 --- /dev/null +++ b/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/oap/OapOutputWriter.scala @@ -0,0 +1,64 @@ +/* + * 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.datasources.oap + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.{OutputWriter, WriteResult} +import org.apache.spark.sql.execution.datasources.oap.io.OapDataWriter +import org.apache.spark.sql.types.StructType + + +private[oap] class OapOutputWriter( + path: String, + dataSchema: StructType, + context: TaskAttemptContext) extends OutputWriter { + private var rowCount = 0 + private var partitionString: String = "" + + override def setPartitionString(ps: String): Unit = { + partitionString = ps + } + + private val writer: OapDataWriter = { + val isCompressed = FileOutputFormat.getCompressOutput(context) + val conf = context.getConfiguration + val file: Path = new Path(path) + val fs = file.getFileSystem(conf) + val fileOut = fs.create(file, false) + + new OapDataWriter(isCompressed, fileOut, dataSchema, conf) + } + + override def write(row: InternalRow): Unit = { + rowCount += 1 + writer.write(row) + } + + override def close(): Unit = { + writer.close() + } + + override def writeStatus(): WriteResult = { + OapWriteResult(dataFileName, rowCount, partitionString) + } + + def dataFileName: String = new Path(path).getName +} diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexOutputWriter.scala b/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexOutputWriter.scala new file mode 100644 index 00000000000..b5ee8487688 --- /dev/null +++ b/src/main/spark2.2/scala/org/apache/spark/sql/execution/datasources/oap/index/OapIndexOutputWriter.scala @@ -0,0 +1,87 @@ +/* + * 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.datasources.oap.index + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.{RecordWriter, TaskAttemptContext} +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat +import org.apache.parquet.hadoop.util.ContextUtil + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.OutputWriter +import org.apache.spark.sql.oap.adapter.InputFileNameHolderAdapter + +// TODO: parameter name "path" is ambiguous +private[index] class OapIndexOutputWriter( + path: String, + context: TaskAttemptContext +) extends OutputWriter { + + private val outputFormat = new OapIndexOutputFormat() { + override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { + val outputPath = FileOutputFormat.getOutputPath(context) + val configuration = ContextUtil.getConfiguration(context) + IndexUtils.generateTempIndexFilePath( + configuration, inputFileName, outputPath, path, extension) + } + } + + private var recordWriter: RecordWriter[Void, InternalRow] = _ + + private var inputFileName: String = _ + + private var results: Seq[IndexBuildResult] = Nil + + private var rowCount: Long = 0 + + override def write(row: InternalRow): Unit = { + checkStartOfNewFile() + recordWriter.write(null, row) + rowCount += 1 + } + + override def close(): Unit = { + closeWriter() + } + + override def writeStatus(): Seq[IndexBuildResult] = { + results + } + + private def initWriter(): Unit = { + inputFileName = InputFileNameHolderAdapter.getInputFileName().toString + recordWriter = outputFormat.getRecordWriter(context) + rowCount = 0 + } + + private def closeWriter(): Unit = { + if (recordWriter != null) { + recordWriter.close(context) + recordWriter = null + results = results :+ IndexBuildResult( + new Path(inputFileName).getName, rowCount, "", new Path(inputFileName).getParent.toString) + } + } + + private def checkStartOfNewFile(): Unit = { + if (inputFileName != InputFileNameHolderAdapter.getInputFileName().toString) { + closeWriter() + initWriter() + } + } +} diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/hive/OapSessionStateBuilder.scala b/src/main/spark2.2/scala/org/apache/spark/sql/hive/OapSessionStateBuilder.scala new file mode 100644 index 00000000000..240b088f97e --- /dev/null +++ b/src/main/spark2.2/scala/org/apache/spark/sql/hive/OapSessionStateBuilder.scala @@ -0,0 +1,72 @@ +/* + * 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.hive + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.parser.{ParserInterface, SqlBaseParser} +import org.apache.spark.sql.execution.{SparkPlanner, SparkSqlParser} +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FileSourceStrategy} +import org.apache.spark.sql.execution.datasources.oap.OapStrategies +import org.apache.spark.sql.internal.{SQLConf, SessionState, VariableSubstitution} + +class OapSessionStateBuilder(session: SparkSession, parentState: Option[SessionState] = None) + extends HiveSessionStateBuilder(session, parentState) { + + override lazy val sqlParser: ParserInterface = new OapSqlParser(conf) + + override protected def planner: SparkPlanner = { + new SparkPlanner(session.sparkContext, conf, experimentalMethods) + with HiveStrategies + with OapStrategies { + override val sparkSession: SparkSession = session + + override def extraPlanningStrategies: Seq[Strategy] = + super.extraPlanningStrategies ++ customPlanningStrategies + + override def strategies: Seq[Strategy] = { + experimentalMethods.extraStrategies ++ + extraPlanningStrategies ++ + oapStrategies ++ Seq( + FileSourceStrategy, + DataSourceStrategy(conf), + SpecialLimits, + InMemoryScans, + HiveTableScans, + Scripts, + Aggregation, + JoinSelection, + BasicOperators + ) + } + } + } + + override protected def newBuilder: NewBuilder = new OapSessionStateBuilder(_, _) +} + + +/** + * Concrete parser for Spark SQL statements. + */ +class OapSqlParser(conf: SQLConf) extends SparkSqlParser(conf) { + private val substitutor = new VariableSubstitution(conf) + + protected override def parse[T](command: String)(toResult: SqlBaseParser => T): T = { + super.parse(substitutor.substitute(command))(toResult) + } +} diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/src/main/spark2.2/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 5e4734ad3ad..ee5e9f749c1 100644 --- a/src/main/spark2.2/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/src/main/spark2.2/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -78,10 +78,10 @@ object HiveThriftServer2 extends Logging { optionsProcessor.parse(args) logInfo("Starting SparkContext") - SparkSQLEnv.init() + OapEnv.init() ShutdownHookManager.addShutdownHook { () => - SparkSQLEnv.stop() + OapEnv.stop() uiTab.foreach(_.detach()) } diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/hive/thriftserver/OapEnv.scala b/src/main/spark2.2/scala/org/apache/spark/sql/hive/thriftserver/OapEnv.scala new file mode 100644 index 00000000000..43b45b5dcf6 --- /dev/null +++ b/src/main/spark2.2/scala/org/apache/spark/sql/hive/thriftserver/OapEnv.scala @@ -0,0 +1,79 @@ +/* + * 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.hive.thriftserver + +import java.io.PrintStream + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} +import org.apache.spark.sql.oap.OapSession +import org.apache.spark.sql.oap.listener.OapListener +import org.apache.spark.sql.oap.ui.OapTab +import org.apache.spark.util.Utils + +/** + * Most of the code in init() are copied from SparkSQLEnv. Please include code from the + * corresponding Spark version. + */ +private[hive] object OapEnv extends Logging { + logDebug("Initializing Oap Env") + + var sqlContext: SQLContext = _ + var sparkContext: SparkContext = _ + + def init() { + if (sqlContext == null) { + val sparkConf = new SparkConf(loadDefaults = true) + // If user doesn't specify the appName, we want to get [SparkSQL::localHostName] instead of + // the default appName [SparkSQLCLIDriver] in cli or beeline. + val maybeAppName = sparkConf + .getOption("spark.app.name") + .filterNot(_ == classOf[SparkSQLCLIDriver].getName) + .filterNot(_ == classOf[HiveThriftServer2].getName) + + sparkConf + .setAppName(maybeAppName.getOrElse(s"SparkSQL::${Utils.localHostName()}")) + + val sparkSession = OapSession.builder.config(sparkConf).enableHiveSupport().getOrCreate() + sparkContext = sparkSession.sparkContext + sqlContext = sparkSession.sqlContext + + val metadataHive = sparkSession + .sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog] + .client.newSession() + metadataHive.setOut(new PrintStream(System.out, true, "UTF-8")) + metadataHive.setInfo(new PrintStream(System.err, true, "UTF-8")) + metadataHive.setError(new PrintStream(System.err, true, "UTF-8")) + sparkSession.conf.set("spark.sql.hive.version", HiveUtils.hiveExecutionVersion) + } + + sparkContext.addSparkListener(new OapListener) + + SparkSQLEnv.sparkContext = sparkContext + SparkSQLEnv.sqlContext = sqlContext + + sparkContext.ui.foreach(new OapTab(_)) + } + + /** Cleans up and shuts down the Spark SQL environments. */ + def stop() { + SparkSQLEnv.stop() + } +} diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/src/main/spark2.2/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 33e18a8da60..b371a913d24 100644 --- a/src/main/spark2.2/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/src/main/spark2.2/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -115,7 +115,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { SessionState.start(sessionState) // Clean up after we exit - ShutdownHookManager.addShutdownHook { () => SparkSQLEnv.stop() } + ShutdownHookManager.addShutdownHook { () => OapEnv.stop() } val remoteMode = isRemoteMode(sessionState) // "-h" option has been passed, so connect to Hive thrift server. @@ -290,16 +290,12 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { // Force initializing SparkSQLEnv. This is put here but not object SparkSQLCliDriver // because the Hive unit tests do not go through the main() code path. if (!isRemoteMode) { - SparkSQLEnv.init() + OapEnv.init() } else { // Hive 1.2 + not supported in CLI throw new RuntimeException("Remote operations not supported") } - override def setHiveVariables(hiveVariables: java.util.Map[String, String]): Unit = { - hiveVariables.asScala.foreach(kv => SparkSQLEnv.sqlContext.conf.setConfString(kv._1, kv._2)) - } - override def processCmd(cmd: String): Int = { val cmd_trimmed: String = cmd.trim() val cmd_lower = cmd_trimmed.toLowerCase(Locale.ROOT) diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/oap/OapSession.scala b/src/main/spark2.2/scala/org/apache/spark/sql/oap/OapSession.scala new file mode 100644 index 00000000000..21eea0e04ed --- /dev/null +++ b/src/main/spark2.2/scala/org/apache/spark/sql/oap/OapSession.scala @@ -0,0 +1,360 @@ +/* + * 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.oap + +import java.util.concurrent.atomic.AtomicReference + +import scala.util.control.NonFatal + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} +import org.apache.spark.sql.{SparkSession, SparkSessionExtensions} +import org.apache.spark.sql.execution.ui.SQLListener +import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SessionState, SessionStateBuilder, StaticSQLConf} +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION +import org.apache.spark.util.Utils + +class OapSession(sparkContext: SparkContext) extends SparkSession(sparkContext) { self => + @InterfaceStability.Unstable + @transient + override lazy val sessionState: SessionState = { + OapSession.instantiateSessionState( + OapSession.sessionStateClassName(sparkContext.conf), + self) + } +} + +object OapSession { + class OapSessionBuilder extends SparkSession.Builder { + private[this] val options = new scala.collection.mutable.HashMap[String, String] + + private[this] val extensions = new SparkSessionExtensions + + private[this] var userSuppliedContext: Option[SparkContext] = None + + override private[spark] def sparkContext(sparkContext: SparkContext): OapSessionBuilder = synchronized { + userSuppliedContext = Option(sparkContext) + this + } + + /** + * Sets a name for the application, which will be shown in the Spark web UI. + * If no application name is set, a randomly generated name will be used. + * + * @since 2.0.0 + */ + override def appName(name: String): OapSessionBuilder = config("spark.app.name", name) + + /** + * Sets a config option. Options set using this method are automatically propagated to + * both `SparkConf` and SparkSession's own configuration. + * + * @since 2.0.0 + */ + override def config(key: String, value: String): OapSessionBuilder = synchronized { + options += key -> value + this + } + + /** + * Sets a config option. Options set using this method are automatically propagated to + * both `SparkConf` and SparkSession's own configuration. + * + * @since 2.0.0 + */ + override def config(key: String, value: Long): OapSessionBuilder = synchronized { + options += key -> value.toString + this + } + + /** + * Sets a config option. Options set using this method are automatically propagated to + * both `SparkConf` and SparkSession's own configuration. + * + * @since 2.0.0 + */ + override def config(key: String, value: Double): OapSessionBuilder = synchronized { + options += key -> value.toString + this + } + + /** + * Sets a config option. Options set using this method are automatically propagated to + * both `SparkConf` and SparkSession's own configuration. + * + * @since 2.0.0 + */ + override def config(key: String, value: Boolean): OapSessionBuilder = synchronized { + options += key -> value.toString + this + } + + /** + * Sets a list of config options based on the given `SparkConf`. + * + * @since 2.0.0 + */ + override def config(conf: SparkConf): OapSessionBuilder = synchronized { + conf.getAll.foreach { case (k, v) => options += k -> v } + this + } + + /** + * Sets the Spark master URL to connect to, such as "local" to run locally, "local[4]" to + * run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster. + * + * @since 2.0.0 + */ + override def master(master: String): OapSessionBuilder = config("spark.master", master) + + /** + * Enables Hive support, including connectivity to a persistent Hive metastore, support for + * Hive serdes, and Hive user-defined functions. + * + * @since 2.0.0 + */ + override def enableHiveSupport(): OapSessionBuilder = synchronized { + if (hiveClassesArePresent) { + config(CATALOG_IMPLEMENTATION.key, "hive") + } else { + throw new IllegalArgumentException( + "Unable to instantiate SparkSession with Hive support because " + + "Hive classes are not found.") + } + } + + /** + * Inject extensions into the [[SparkSession]]. This allows a user to add Analyzer rules, + * Optimizer rules, Planning Strategies or a customized parser. + * + * @since 2.2.0 + */ + override def withExtensions(f: SparkSessionExtensions => Unit): OapSessionBuilder = { + f(extensions) + this + } + + /** + * Gets an existing [[SparkSession]] or, if there is no existing one, creates a new + * one based on the options set in this builder. + * + * This method first checks whether there is a valid thread-local SparkSession, + * and if yes, return that one. It then checks whether there is a valid global + * default SparkSession, and if yes, return that one. If no valid global default + * SparkSession exists, the method creates a new SparkSession and assigns the + * newly created SparkSession as the global default. + * + * In case an existing SparkSession is returned, the config options specified in + * this builder will be applied to the existing SparkSession. + * + * @since 2.0.0 + */ + override def getOrCreate(): OapSession = synchronized { + // Get the session from current thread's active session. + var session = activeThreadSession.get() + if ((session ne null) && !session.sparkContext.isStopped) { + options.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) } + if (options.nonEmpty) { + logWarning("Using an existing SparkSession; some configuration may not take effect.") + } + return session + } + + // Global synchronization so we will only set the default session once. + OapSession.synchronized { + // If the current thread does not have an active session, get it from the global session. + session = defaultSession.get() + if ((session ne null) && !session.sparkContext.isStopped) { + options.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) } + if (options.nonEmpty) { + logWarning("Using an existing SparkSession; some configuration may not take effect.") + } + return session + } + + // No active nor global default session. Create a new one. + val sparkContext = userSuppliedContext.getOrElse { + // set app name if not given + val randomAppName = java.util.UUID.randomUUID().toString + val sparkConf = new SparkConf() + options.foreach { case (k, v) => sparkConf.set(k, v) } + if (!sparkConf.contains("spark.app.name")) { + sparkConf.setAppName(randomAppName) + } + val sc = SparkContext.getOrCreate(sparkConf) + // maybe this is an existing SparkContext, update its SparkConf which maybe used + // by SparkSession + options.foreach { case (k, v) => sc.conf.set(k, v) } + if (!sc.conf.contains("spark.app.name")) { + sc.conf.setAppName(randomAppName) + } + sc + } + + // Initialize extensions if the user has defined a configurator class. + val extensionConfOption = sparkContext.conf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS) + if (extensionConfOption.isDefined) { + val extensionConfClassName = extensionConfOption.get + try { + val extensionConfClass = Utils.classForName(extensionConfClassName) + val extensionConf = extensionConfClass.newInstance() + .asInstanceOf[SparkSessionExtensions => Unit] + extensionConf(extensions) + } catch { + // Ignore the error if we cannot find the class or when the class has the wrong type. + case e @ (_: ClassCastException | + _: ClassNotFoundException | + _: NoClassDefFoundError) => + logWarning(s"Cannot use $extensionConfClassName to configure session extensions.", e) + } + } + + session = new OapSession(sparkContext) + options.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) } + defaultSession.set(session) + + // Register a successfully instantiated context to the singleton. This should be at the + // end of the class definition so that the singleton is updated only if there is no + // exception in the construction of the instance. + sparkContext.addSparkListener(new SparkListener { + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { + defaultSession.set(null) + sqlListener.set(null) + } + }) + } + + return session + } + } + + /** + * Creates a [[SparkSession.Builder]] for constructing a [[SparkSession]]. + * + * @since 2.0.0 + */ + def builder(): OapSessionBuilder = new OapSessionBuilder + + /** + * Changes the SparkSession that will be returned in this thread and its children when + * SparkSession.getOrCreate() is called. This can be used to ensure that a given thread receives + * a SparkSession with an isolated session, instead of the global (first created) context. + * + * @since 2.0.0 + */ + def setActiveSession(session: OapSession): Unit = { + activeThreadSession.set(session) + } + + /** + * Clears the active SparkSession for current thread. Subsequent calls to getOrCreate will + * return the first created context instead of a thread-local override. + * + * @since 2.0.0 + */ + def clearActiveSession(): Unit = { + activeThreadSession.remove() + } + + /** + * Sets the default SparkSession that is returned by the builder. + * + * @since 2.0.0 + */ + def setDefaultSession(session: OapSession): Unit = { + defaultSession.set(session) + } + + /** + * Clears the default SparkSession that is returned by the builder. + * + * @since 2.0.0 + */ + def clearDefaultSession(): Unit = { + defaultSession.set(null) + } + + /** + * Returns the active SparkSession for the current thread, returned by the builder. + * + * @since 2.2.0 + */ + def getActiveSession: Option[OapSession] = Option(activeThreadSession.get) + + /** + * Returns the default SparkSession that is returned by the builder. + * + * @since 2.2.0 + */ + def getDefaultSession: Option[OapSession] = Option(defaultSession.get) + + /** A global SQL listener used for the SQL UI. */ + private[sql] val sqlListener = new AtomicReference[SQLListener]() + + //////////////////////////////////////////////////////////////////////////////////////// + // Private methods from now on + //////////////////////////////////////////////////////////////////////////////////////// + + /** The active SparkSession for the current thread. */ + private val activeThreadSession = new InheritableThreadLocal[OapSession] + + /** Reference to the root SparkSession. */ + private val defaultSession = new AtomicReference[OapSession] + + private val HIVE_SESSION_STATE_BUILDER_CLASS_NAME = + "org.apache.spark.sql.hive.OapSessionStateBuilder" + + private def sessionStateClassName(conf: SparkConf): String = { + conf.get(CATALOG_IMPLEMENTATION) match { + case "hive" => HIVE_SESSION_STATE_BUILDER_CLASS_NAME + case "in-memory" => classOf[SessionStateBuilder].getCanonicalName + } + } + + /** + * Helper method to create an instance of `SessionState` based on `className` from conf. + * The result is either `SessionState` or a Hive based `SessionState`. + */ + private def instantiateSessionState( + className: String, + oapSession: OapSession): SessionState = { + try { + // invoke `new [Hive]SessionStateBuilder(SparkSession, Option[SessionState])` + val clazz = Utils.classForName(className) + val ctor = clazz.getConstructors.head + ctor.newInstance(oapSession, None).asInstanceOf[BaseSessionStateBuilder].build() + } catch { + case NonFatal(e) => + throw new IllegalArgumentException(s"Error while instantiating '$className':", e) + } + } + + /** + * @return true if Hive classes can be loaded, otherwise false. + */ + private[spark] def hiveClassesArePresent: Boolean = { + try { + Utils.classForName(HIVE_SESSION_STATE_BUILDER_CLASS_NAME) + Utils.classForName("org.apache.hadoop.hive.conf.HiveConf") + true + } catch { + case _: ClassNotFoundException | _: NoClassDefFoundError => false + } + } +} diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/AggregateFunctionAdapter.scala b/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/AggregateFunctionAdapter.scala new file mode 100644 index 00000000000..90b4e9f48ba --- /dev/null +++ b/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/AggregateFunctionAdapter.scala @@ -0,0 +1,25 @@ +/* + * 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.oap.adapter + +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction + +object AggregateFunctionAdapter { + // In Spark 2.2 and later, all aggregation functions support partial aggregate, see SPARK-19060 + def supportsPartial(func: AggregateFunction): Boolean = true +} diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/FileIndexAdapter.scala b/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/FileIndexAdapter.scala new file mode 100644 index 00000000000..c0dc5d5dda2 --- /dev/null +++ b/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/FileIndexAdapter.scala @@ -0,0 +1,33 @@ +/* + * 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.oap.adapter + +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.execution.datasources.{FileIndex, PartitionDirectory} + +object FileIndexAdapter { + /** + * Parameter dataFilters is added in Spark 2.2 and later. In 2.1, we just ignore it. + */ + def listFiles( + fileIndex: FileIndex, + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { + fileIndex.listFiles(partitionFilters, dataFilters) + } +} diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/FileSourceScanExecAdapter.scala b/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/FileSourceScanExecAdapter.scala new file mode 100644 index 00000000000..9b8455bc14f --- /dev/null +++ b/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/FileSourceScanExecAdapter.scala @@ -0,0 +1,43 @@ +/* + * 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.oap.adapter + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute} +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation} +import org.apache.spark.sql.types.StructType + +object FileSourceScanExecAdapter { + def createFileSourceScanExec( + relation: HadoopFsRelation, + output: Seq[Attribute], + requiredSchema: StructType, + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression], + metastoreTableIdentifier: Option[TableIdentifier]): FileSourceScanExec = { + FileSourceScanExec( + relation, + output, + requiredSchema, + partitionFilters, + dataFilters, + metastoreTableIdentifier) + } +} diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/InputFileNameHolderAdapter.scala b/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/InputFileNameHolderAdapter.scala new file mode 100644 index 00000000000..525f1eb77e8 --- /dev/null +++ b/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/InputFileNameHolderAdapter.scala @@ -0,0 +1,28 @@ +/* + * 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.oap.adapter + +import org.apache.spark.rdd.InputFileBlockHolder +import org.apache.spark.unsafe.types.UTF8String + +object InputFileNameHolderAdapter { + /** + * Returns the holding file name or empty string if it is unknown. + */ + def getInputFileName(): UTF8String = InputFileBlockHolder.getInputFilePath +} diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/LogicalPlanAdapter.scala b/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/LogicalPlanAdapter.scala new file mode 100644 index 00000000000..28aa836cd6a --- /dev/null +++ b/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/LogicalPlanAdapter.scala @@ -0,0 +1,25 @@ +/* + * 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.oap.adapter + +import org.apache.spark.sql.catalyst.plans.logical.{Statistics, LogicalPlan} +import org.apache.spark.sql.internal.SQLConf + +object LogicalPlanAdapter { + def getStatistics(plan: LogicalPlan, conf:SQLConf): Statistics = plan.stats(conf) +} diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/RpcEndpointRefAdapter.scala b/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/RpcEndpointRefAdapter.scala new file mode 100644 index 00000000000..a27b6c0ef6a --- /dev/null +++ b/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/RpcEndpointRefAdapter.scala @@ -0,0 +1,28 @@ +/* + * 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.oap.adapter + +import scala.reflect.ClassTag + +import org.apache.spark.rpc.RpcEndpointRef + +object RpcEndpointRefAdapter { + def askSync[T: ClassTag](endpoint: RpcEndpointRef, message: Any): T = { + endpoint.askSync(message) + } +} diff --git a/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/SqlConfAdapter.scala b/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/SqlConfAdapter.scala new file mode 100644 index 00000000000..1a067eb5435 --- /dev/null +++ b/src/main/spark2.2/scala/org/apache/spark/sql/oap/adapter/SqlConfAdapter.scala @@ -0,0 +1,25 @@ +/* + * 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.oap.adapter + +import org.apache.spark.internal.config.ConfigBuilder +import org.apache.spark.sql.internal.SQLConf + +object SqlConfAdapter { + def buildConf(key: String): ConfigBuilder = SQLConf.buildConf(key) +} diff --git a/src/main/spark2.2/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/src/main/spark2.2/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index f17b6377548..ce0d7bd7435 100644 --- a/src/main/spark2.2/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/src/main/spark2.2/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -227,6 +227,18 @@ private[v1] class ApiRootResource extends ApiRequestContext { new ApplicationEnvironmentResource(ui) } } + + /** + * Added for OAP FiberCacheManager Metrics + * @param appId + * @return impl instance + */ + @Path("applications/{appId}/fibercachemanagers") + def getCacheManagers(@PathParam("appId") appId: String): AllFiberCacheManagerListResource = { + withSparkUI(appId, None) { ui => + new AllFiberCacheManagerListResource(ui) + } + } } private[spark] object ApiRootResource { diff --git a/src/main/spark2.2/scala/org/apache/spark/storage/BlockId.scala b/src/main/spark2.2/scala/org/apache/spark/storage/BlockId.scala index 524f6970992..463bd193338 100644 --- a/src/main/spark2.2/scala/org/apache/spark/storage/BlockId.scala +++ b/src/main/spark2.2/scala/org/apache/spark/storage/BlockId.scala @@ -100,6 +100,10 @@ private[spark] case class TestBlockId(id: String) extends BlockId { override def name: String = "test_" + id } +private[spark] case class FiberBlockId(id: String) extends BlockId { + override def name: String = "fiber_" + id +} + @DeveloperApi object BlockId { val RDD = "rdd_([0-9]+)_([0-9]+)".r @@ -110,6 +114,7 @@ object BlockId { val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r val TEST = "test_(.*)".r + val FIBER = "fiber_(.*)".r /** Converts a BlockId "name" String back into a BlockId. */ def apply(id: String): BlockId = id match { @@ -129,6 +134,8 @@ object BlockId { StreamBlockId(streamId.toInt, uniqueId.toLong) case TEST(value) => TestBlockId(value) + case FIBER(value) => + FiberBlockId(value) case _ => throw new IllegalStateException("Unrecognized BlockId: " + id) } diff --git a/src/main/spark2.2/scala/org/apache/spark/util/collection/BitSet.scala b/src/main/spark2.2/scala/org/apache/spark/util/collection/BitSet.scala index e63e0e3e1f6..d265db14ea0 100644 --- a/src/main/spark2.2/scala/org/apache/spark/util/collection/BitSet.scala +++ b/src/main/spark2.2/scala/org/apache/spark/util/collection/BitSet.scala @@ -34,6 +34,15 @@ class BitSet(numBits: Int) extends Serializable { */ def capacity: Int = numWords * 64 + def toLongArray(): Array[Long] = words + + def this(from: Array[Long]) { + this(from.length << 6) + for(i <- from.indices) { + words(i) = from(i) + } + } + /** * Clear all set bits. */ diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index 014b094f49f..3946ae7d25d 100644 --- a/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -41,7 +41,7 @@ import org.apache.spark.util.Utils class FileSourceStrategySuite extends QueryTest with SharedOapContext with PredicateHelper { import testImplicits._ - sparkConf.set("spark.default.parallelism", "1") + oapSparkConf.set("spark.default.parallelism", "1") // Override afterEach because we don't want to check open streams override def beforeEach(): Unit = {} @@ -398,7 +398,7 @@ class FileSourceStrategySuite extends QueryTest with SharedOapContext with Predi val fileCatalog = new InMemoryFileIndex( sparkSession = spark, - rootPaths = Seq(new Path(tempDir)), + Seq(new Path(tempDir)), parameters = Map.empty[String, String], partitionSchema = None) // This should not fail. diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/FilterSuite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/FilterSuite.scala index 844c9cb2f6e..f0e5446a2d2 100644 --- a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/FilterSuite.scala +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/FilterSuite.scala @@ -41,12 +41,12 @@ class FilterSuite extends QueryTest with SharedOapContext with BeforeAndAfterEac override def beforeAll(): Unit = { super.beforeAll() // In this suite we don't want to skip index even if the cost is higher. - defaultEis = sqlConf.getConf(OapConf.OAP_ENABLE_EXECUTOR_INDEX_SELECTION) - sqlConf.setConf(OapConf.OAP_ENABLE_EXECUTOR_INDEX_SELECTION, false) + defaultEis = sqlContext.conf.getConf(OapConf.OAP_ENABLE_EXECUTOR_INDEX_SELECTION) + sqlContext.conf.setConf(OapConf.OAP_ENABLE_EXECUTOR_INDEX_SELECTION, false) } override def afterAll(): Unit = { - sqlConf.setConf(OapConf.OAP_ENABLE_EXECUTOR_INDEX_SELECTION, defaultEis) + sqlContext.conf.setConf(OapConf.OAP_ENABLE_EXECUTOR_INDEX_SELECTION, defaultEis) super.afterAll() } @@ -100,9 +100,9 @@ class FilterSuite extends QueryTest with SharedOapContext with BeforeAndAfterEac } test("test oap row group size change") { - val previousRowGroupSize = sqlConf.getConfString(OapConf.OAP_ROW_GROUP_SIZE.key) + val previousRowGroupSize = sqlContext.conf.getConfString(OapConf.OAP_ROW_GROUP_SIZE.key) // change default row group size - sqlConf.setConfString(OapConf.OAP_ROW_GROUP_SIZE.key, "1025") + sqlContext.conf.setConfString(OapConf.OAP_ROW_GROUP_SIZE.key, "1025") val data: Seq[(Int, String)] = (1 to 3000).map { i => (i, s"this is test $i") } data.toDF("key", "value").createOrReplaceTempView("t") checkAnswer(sql("SELECT * FROM oap_test"), Seq.empty[Row]) @@ -110,10 +110,10 @@ class FilterSuite extends QueryTest with SharedOapContext with BeforeAndAfterEac checkAnswer(sql("SELECT * FROM oap_test"), data.map { row => Row(row._1, row._2) }) // set back to default value if (previousRowGroupSize == null) { - sqlConf.setConfString(OapConf.OAP_ROW_GROUP_SIZE.key, + sqlContext.conf.setConfString(OapConf.OAP_ROW_GROUP_SIZE.key, OapConf.OAP_ROW_GROUP_SIZE.defaultValueString) } else { - sqlConf.setConfString(OapConf.OAP_ROW_GROUP_SIZE.key, + sqlContext.conf.setConfString(OapConf.OAP_ROW_GROUP_SIZE.key, previousRowGroupSize) } } @@ -351,7 +351,7 @@ class FilterSuite extends QueryTest with SharedOapContext with BeforeAndAfterEac sql("refresh oindex on t_refresh_parquet partition (b=2)") val fs = new Path(currentPath).getFileSystem(new Configuration()) - val tablePath = sqlConf.warehousePath + "/t_refresh_parquet/" + val tablePath = sqlContext.conf.warehousePath + "/t_refresh_parquet/" assert(fs.globStatus(new Path(tablePath + "b=1/*.index")).length == 1) assert(fs.globStatus(new Path(tablePath + "b=2/*.index")).length == 2) assert(fs.globStatus(new Path(tablePath + "b=3/*.index")).length == 0) @@ -470,7 +470,7 @@ class FilterSuite extends QueryTest with SharedOapContext with BeforeAndAfterEac sql("refresh oindex on t_refresh partition (b=2)") val fs = new Path(currentPath).getFileSystem(new Configuration()) - val tablePath = sqlConf.warehousePath + "/t_refresh/" + val tablePath = sqlContext.conf.warehousePath + "/t_refresh/" assert(fs.globStatus(new Path(tablePath + "b=1/*.index")).length == 1) assert(fs.globStatus(new Path(tablePath + "b=2/*.index")).length == 2) assert(fs.globStatus(new Path(tablePath + "b=3/*.index")).length == 0) diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapCheckIndexSuite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapCheckIndexSuite.scala index 76e46b461ee..494b655cba3 100644 --- a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapCheckIndexSuite.scala +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapCheckIndexSuite.scala @@ -104,7 +104,7 @@ class OapCheckIndexSuite extends QueryTest with SharedOapContext with BeforeAndA checkAnswer(sql("check oindex on oap_partition_table"), Nil) val partitionPath = - new Path(sqlConf.warehousePath + "/oap_partition_table/b=2/c=c2") + new Path(sqlContext.conf.warehousePath + "/oap_partition_table/b=2/c=c2") Utils.deleteRecursively(new File(partitionPath.toUri.getPath, OapFileFormat.OAP_META_FILE)) checkAnswer(sql("check oindex on oap_partition_table"), @@ -203,8 +203,8 @@ class OapCheckIndexSuite extends QueryTest with SharedOapContext with BeforeAndA |SELECT key from t where value == 4 """.stripMargin) - val path1 = new Path(sqlConf.warehousePath + "/oap_partition_table/b=1/c=c1") - val path2 = new Path(sqlConf.warehousePath + "/oap_partition_table/b=2/c=c2") + val path1 = new Path(sqlContext.conf.warehousePath + "/oap_partition_table/b=1/c=c1") + val path2 = new Path(sqlContext.conf.warehousePath + "/oap_partition_table/b=2/c=c2") checkAnswer(sql("check oindex on oap_partition_table"), Seq(Row(s"Meta file not found in partition: ${path1.toUri.getPath}"), @@ -240,7 +240,7 @@ class OapCheckIndexSuite extends QueryTest with SharedOapContext with BeforeAndA checkAnswer(sql("check oindex on oap_partition_table"), Nil) // Delete a data file - val partitionPath = new Path(sqlConf.warehousePath + "/oap_partition_table/b=2/c=c2") + val partitionPath = new Path(sqlContext.conf.warehousePath + "/oap_partition_table/b=2/c=c2") val metaOpt = OapUtils.getMeta(sparkContext.hadoopConfiguration, partitionPath) assert(metaOpt.nonEmpty) assert(metaOpt.get.fileMetas.nonEmpty) @@ -278,7 +278,7 @@ class OapCheckIndexSuite extends QueryTest with SharedOapContext with BeforeAndA checkAnswer(sql("check oindex on oap_partition_table"), Nil) // Delete an index file - val partitionPath = new Path(sqlConf.warehousePath + "/oap_partition_table/b=2/c=c2") + val partitionPath = new Path(sqlContext.conf.warehousePath + "/oap_partition_table/b=2/c=c2") val metaOpt = OapUtils.getMeta(sparkContext.hadoopConfiguration, partitionPath) assert(metaOpt.nonEmpty) assert(metaOpt.get.fileMetas.nonEmpty) @@ -322,8 +322,8 @@ class OapCheckIndexSuite extends QueryTest with SharedOapContext with BeforeAndA |SELECT key from t where value == 104 """.stripMargin) - val path1 = new Path(sqlConf.warehousePath + "/oap_partition_table/b=1/c=c1") - val path2 = new Path(sqlConf.warehousePath + "/oap_partition_table/b=2/c=c2") + val path1 = new Path(sqlContext.conf.warehousePath + "/oap_partition_table/b=1/c=c1") + val path2 = new Path(sqlContext.conf.warehousePath + "/oap_partition_table/b=2/c=c2") checkAnswer(sql("check oindex on oap_partition_table"), Seq(Row(s"Meta file not found in partition: ${path1.toUri.getPath}"), @@ -367,7 +367,7 @@ class OapCheckIndexSuite extends QueryTest with SharedOapContext with BeforeAndA """.stripMargin) val partitionPath = - new Path(sqlConf.warehousePath + "/oap_partition_table/b=2/c=c2") + new Path(sqlContext.conf.warehousePath + "/oap_partition_table/b=2/c=c2") checkAnswer( sql("check oindex on oap_partition_table partition(b=2, c='c2')"), Row(s"Meta file not found in partition: ${partitionPath.toUri.getPath}")) @@ -413,7 +413,7 @@ class OapCheckIndexSuite extends QueryTest with SharedOapContext with BeforeAndA checkAnswer(sql("check oindex on oap_partition_table partition(b=2, c='c2')"), Nil) val partitionPath = - new Path(sqlConf.warehousePath + "/oap_partition_table/b=2/c=c2") + new Path(sqlContext.conf.warehousePath + "/oap_partition_table/b=2/c=c2") // Delete a data file val metaOpt = OapUtils.getMeta(sparkContext.hadoopConfiguration, partitionPath) assert(metaOpt.nonEmpty) @@ -453,7 +453,7 @@ class OapCheckIndexSuite extends QueryTest with SharedOapContext with BeforeAndA checkAnswer(sql("check oindex on oap_partition_table partition(b=2, c='c2')"), Nil) // Delete an index file - val partitionPath = new Path(sqlConf.warehousePath + "/oap_partition_table/b=2/c=c2") + val partitionPath = new Path(sqlContext.conf.warehousePath + "/oap_partition_table/b=2/c=c2") val metaOpt = OapUtils.getMeta(sparkContext.hadoopConfiguration, partitionPath) assert(metaOpt.nonEmpty) assert(metaOpt.get.fileMetas.nonEmpty) diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapDDLSuite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapDDLSuite.scala index cb801a7ac26..6cef4aded44 100644 --- a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapDDLSuite.scala +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapDDLSuite.scala @@ -103,7 +103,7 @@ class OapDDLSuite extends QueryTest with SharedOapContext with BeforeAndAfterEac val data: Seq[(Int, Int)] = (1 to 10).map { i => (i, i) } data.toDF("key", "value").createOrReplaceTempView("t") - val path = new Path(sqlConf.warehousePath) + val path = new Path(sqlContext.conf.warehousePath) sql( """ diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapIndexQuerySuite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapIndexQuerySuite.scala index a9d6ef48eb8..8f54e5a5403 100644 --- a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapIndexQuerySuite.scala +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapIndexQuerySuite.scala @@ -131,7 +131,7 @@ class OapIndexQuerySuite extends QueryTest with SharedOapContext with BeforeAndA } test("startswith using index") { - sqlConf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, false) + sqlContext.conf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, false) val data: Seq[(Int, String)] = scala.util.Random.shuffle(1 to 30).map(i => (i, s"this$i is test")) data.toDF("key", "value").createOrReplaceTempView("t") @@ -141,11 +141,11 @@ class OapIndexQuerySuite extends QueryTest with SharedOapContext with BeforeAndA checkAnswer(sql("SELECT * FROM oap_test_1 WHERE b like 'this3%'"), Row(3, "this3 is test") :: Row(30, "this30 is test") :: Nil) } - sqlConf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, true) + sqlContext.conf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, true) } test("startswith using multi-dimension index") { - sqlConf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, false) + sqlContext.conf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, false) val data: Seq[(Int, String)] = scala.util.Random.shuffle(1 to 30).map(i => (i, s"this$i is test")) data.toDF("key", "value").createOrReplaceTempView("t") @@ -155,11 +155,11 @@ class OapIndexQuerySuite extends QueryTest with SharedOapContext with BeforeAndA checkAnswer(sql("SELECT * FROM oap_test_1 WHERE b like 'this3%'"), Row(3, "this3 is test") :: Row(30, "this30 is test") :: Nil) } - sqlConf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, true) + sqlContext.conf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, true) } test("startswith using multi-dimension index - multi-filters") { - sqlConf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, false) + sqlContext.conf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, false) val data: Seq[(Int, String)] = scala.util.Random.shuffle(1 to 30).map(i => (i % 7, s"this$i is test")) data.toDF("key", "value").createOrReplaceTempView("t") @@ -169,11 +169,11 @@ class OapIndexQuerySuite extends QueryTest with SharedOapContext with BeforeAndA checkAnswer(sql("SELECT * FROM oap_test_1 WHERE a = 3 and b like 'this3%'"), Row(3, "this3 is test") :: Nil) } - sqlConf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, true) + sqlContext.conf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, true) } test("startswith using multi-dimension index 2") { - sqlConf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, false) + sqlContext.conf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, false) val data: Seq[(Int, String)] = Seq( 15, 29, 26, 4, 28, 17, 16, 11, 12, 27, 22, 6, 10, 18, 19, 20, 30, 21, 14, 25, 1, 2, 13, 23, 7, 24, 3, 8, 5, 9).map(i => (i, s"this$i is test")) @@ -184,6 +184,6 @@ class OapIndexQuerySuite extends QueryTest with SharedOapContext with BeforeAndA checkAnswer(sql("SELECT * FROM oap_test_1 WHERE b like 'this3%'"), Row(3, "this3 is test") :: Row(30, "this30 is test") :: Nil) } - sqlConf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, true) + sqlContext.conf.setConf(OapConf.OAP_EXECUTOR_INDEX_SELECTION_STATISTICS_POLICY, true) } } diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapMetricsSuite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapMetricsSuite.scala index 070622ab08b..abfb9415969 100644 --- a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapMetricsSuite.scala +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapMetricsSuite.scala @@ -177,7 +177,7 @@ class OapMetricsSuite extends QueryTest with SharedOapContext with BeforeAndAfte sql("create oindex idx1 on parquet_test (a)") // satisfy indexFile / dataFile > 0.7 - sqlConf.setConfString(OapConf.OAP_EXECUTOR_INDEX_SELECTION_FILE_POLICY.key, "true") + sqlContext.conf.setConfString(OapConf.OAP_EXECUTOR_INDEX_SELECTION_FILE_POLICY.key, "true") // SQL 4: ignore index => readBehavior return FULL_SCAN val df = sql("SELECT * FROM parquet_test WHERE a = 1") @@ -190,7 +190,7 @@ class OapMetricsSuite extends QueryTest with SharedOapContext with BeforeAndAfte Some(TaskMetrics(3, 0, 0, 3, 0))) sql("drop oindex idx1 on parquet_test") - sqlConf.setConfString(OapConf.OAP_EXECUTOR_INDEX_SELECTION_FILE_POLICY.key, "false") + sqlContext.conf.setConfString(OapConf.OAP_EXECUTOR_INDEX_SELECTION_FILE_POLICY.key, "false") } test("test OAP accumulators on Oap when miss index") { diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapSuite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapSuite.scala index 3e19b9126b2..e5660a09fef 100644 --- a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapSuite.scala +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/OapSuite.scala @@ -74,15 +74,15 @@ class OapSuite extends QueryTest with SharedOapContext with BeforeAndAfter { test("No Lease Exception on Parquet File Format in Index Building (#243)") { val df = sqlContext.read.format("parquet").load(parquetPath.getAbsolutePath) df.createOrReplaceTempView("parquet_table") - val defaultMaxBytes = sqlConf.getConf(SQLConf.FILES_MAX_PARTITION_BYTES) + val defaultMaxBytes = sqlContext.conf.getConf(SQLConf.FILES_MAX_PARTITION_BYTES) val maxPartitionBytes = 100L - sqlConf.setConf(SQLConf.FILES_MAX_PARTITION_BYTES, maxPartitionBytes) + sqlContext.conf.setConf(SQLConf.FILES_MAX_PARTITION_BYTES, maxPartitionBytes) val numTasks = sql("select * from parquet_table").queryExecution.toRdd.partitions.length withIndex(TestIndex("parquet_table", "parquet_idx")) { sql("create oindex parquet_idx on parquet_table (a)") assert(numTasks == parquetPath.listFiles().filter(_.getName.endsWith(".parquet")) .map(f => Math.ceil(f.length().toDouble / maxPartitionBytes).toInt).sum) - sqlConf.setConf(SQLConf.FILES_MAX_PARTITION_BYTES, defaultMaxBytes) + sqlContext.conf.setConf(SQLConf.FILES_MAX_PARTITION_BYTES, defaultMaxBytes) } } @@ -97,11 +97,11 @@ class OapSuite extends QueryTest with SharedOapContext with BeforeAndAfter { "lZo", "UNCOMPRESSED", "UnCompressed").foreach { codec => - sqlConf.setConfString(OapConf.OAP_COMPRESSION.key, codec) + sqlContext.conf.setConfString(OapConf.OAP_COMPRESSION.key, codec) val df = sqlContext.read.format("oap").load(path.getAbsolutePath) df.write.format("oap").mode(SaveMode.Overwrite).save(path.getAbsolutePath) val compressionType = - sqlConf.getConfString(OapConf.OAP_COMPRESSION.key).toLowerCase + sqlContext.conf.getConfString(OapConf.OAP_COMPRESSION.key).toLowerCase val fileNameIterator = path.listFiles() for (fileName <- fileNameIterator) { if (fileName.toString.endsWith(OapFileFormat.OAP_DATA_EXTENSION)) { @@ -115,7 +115,8 @@ class OapSuite extends QueryTest with SharedOapContext with BeforeAndAfter { } } // Restore compression type back to default. - sqlConf.setConfString(OapConf.OAP_COMPRESSION.key, OapConf.OAP_COMPRESSION.defaultValueString) + sqlContext.conf.setConfString( + OapConf.OAP_COMPRESSION.key, OapConf.OAP_COMPRESSION.defaultValueString) } test("Enable/disable using OAP index after the index is created already") { diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/filecache/FiberSensorSuite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/filecache/FiberSensorSuite.scala index 3247d63f4b0..77063428c94 100644 --- a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/filecache/FiberSensorSuite.scala +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/filecache/FiberSensorSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources.oap.filecache +import scala.collection.JavaConverters._ + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.scalatest.BeforeAndAfterEach @@ -31,15 +33,17 @@ import org.apache.spark.sql.test.oap.{SharedOapContext, TestIndex} import org.apache.spark.util.Utils import org.apache.spark.util.collection.BitSet -class FiberSensorSuite extends QueryTest with SharedOapContext - with AbstractFiberSensor with BeforeAndAfterEach { +class FiberSensorSuite extends QueryTest with SharedOapContext with BeforeAndAfterEach { import testImplicits._ private var currentPath: String = _ + private var fiberSensor: FiberSensor = _ + override def beforeAll(): Unit = { super.beforeAll() + fiberSensor = OapRuntime.getOrCreate.fiberSensor sparkContext.addSparkListener(new OapListener) } @@ -51,18 +55,23 @@ class FiberSensorSuite extends QueryTest with SharedOapContext | USING oap | OPTIONS (path '$path')""".stripMargin) OapRuntime.getOrCreate.fiberCacheManager.clearAllFibers() - FiberCacheManagerSensor.executorToCacheManager.clear() + fiberSensor.executorToCacheManager.clear() } override def afterEach(): Unit = { sqlContext.dropTempTable("oap_test") } - test("test FiberCacheManagerSensor with sql") { + test("test FiberSensor with sql") { + + def getCacheStats(fiberSensor: FiberSensor): CacheStats = + fiberSensor.executorToCacheManager.asScala.toMap.values.foldLeft( + CacheStats())((sum, cache) => sum + cache) + // make each dataFile has 2 rowGroup. // 3000 columns in total, 2 data file by default, 1500 columns each file. // So, each file will have 2 rowGroup. - sqlConf.setConfString(OapConf.OAP_ROW_GROUP_SIZE.key, "1000") + sqlContext.conf.setConfString(OapConf.OAP_ROW_GROUP_SIZE.key, "1000") // Insert data, build index and query, expected hit-index, range ensure all // row groups are cached. @@ -91,9 +100,9 @@ class FiberSensorSuite extends QueryTest with SharedOapContext // Only one executor in local-mode, each data file has 4 dataFiber(2 cols * 2 rgs/col) // wait for a heartbeat Thread.sleep(20 * 1000) - val summary = FiberCacheManagerSensor.summary() + val summary = getCacheStats(fiberSensor) logWarning(s"Summary1: ${summary.toDebugString}") - assertResult(1)(FiberCacheManagerSensor.executorToCacheManager.size()) + assertResult(1)(fiberSensor.executorToCacheManager.size()) // all data are cached when run another sql. // Expect: 1.hitCount increase; 2.missCount equal @@ -102,9 +111,9 @@ class FiberSensorSuite extends QueryTest with SharedOapContext data.filter(r => r._1 > 200 && r._1 < 2400).map(r => Row(r._1, r._2))) CacheStats.reset Thread.sleep(15 * 1000) - val summary2 = FiberCacheManagerSensor.summary() + val summary2 = getCacheStats(fiberSensor) logWarning(s"Summary2: ${summary2.toDebugString}") - assertResult(1)(FiberCacheManagerSensor.executorToCacheManager.size()) + assertResult(1)(fiberSensor.executorToCacheManager.size()) assert(summary.hitCount < summary2.hitCount) assertResult(summary.missCount)(summary2.missCount) assertResult(summary.dataFiberCount)(summary2.dataFiberCount) @@ -114,18 +123,18 @@ class FiberSensorSuite extends QueryTest with SharedOapContext } } - test("test FiberCacheManagerSensor onCustomInfoUpdate FiberCacheManagerMessager") { + test("test FiberSensor onCustomInfoUpdate FiberCacheManagerMessager") { val host = "0.0.0.0" val execID = "exec1" val messager = "FiberCacheManagerMessager" // Test json empty, no more executor added val listener = new OapListener listener.onOtherEvent(SparkListenerCustomInfoUpdate(host, execID, messager, "")) - assertResult(0)(FiberCacheManagerSensor.executorToCacheManager.size()) + assertResult(0)(fiberSensor.executorToCacheManager.size()) // Test json error, no more executor added listener.onOtherEvent(SparkListenerCustomInfoUpdate(host, execID, messager, "error msg")) - assertResult(0)(FiberCacheManagerSensor.executorToCacheManager.size()) + assertResult(0)(fiberSensor.executorToCacheManager.size()) // Test normal msg CacheStats.reset @@ -134,9 +143,9 @@ class FiberSensorSuite extends QueryTest with SharedOapContext val cacheStats = CacheStats(2, 19, 10, 2, 0, 0, 213, 23, 23, 123131, 2) listener.onOtherEvent(SparkListenerCustomInfoUpdate( host, execID, messager, CacheStats.status(cacheStats, conf))) - assertResult(1)(FiberCacheManagerSensor.executorToCacheManager.size()) + assertResult(1)(fiberSensor.executorToCacheManager.size()) assertResult(cacheStats.toJson)( - FiberCacheManagerSensor.executorToCacheManager.get(execID).toJson) + fiberSensor.executorToCacheManager.get(execID).toJson) } test("test get hosts from FiberSensor") { @@ -153,8 +162,8 @@ class FiberSensorSuite extends QueryTest with SharedOapContext val fcs = Seq(FiberCacheStatus(filePath, bitSet1, groupCount, fieldCount)) val fiberInfo = SparkListenerCustomInfoUpdate(host1, execId1, "OapFiberCacheHeartBeatMessager", CacheStatusSerDe.serialize(fcs)) - this.update(fiberInfo) - assert(this.getHosts(filePath) contains (FiberSensor.OAP_CACHE_HOST_PREFIX + host1 + + fiberSensor.updateLocations(fiberInfo) + assert(fiberSensor.getHosts(filePath) contains (FiberSensor.OAP_CACHE_HOST_PREFIX + host1 + FiberSensor.OAP_CACHE_EXECUTOR_PREFIX + execId1)) // executor2 update @@ -171,8 +180,8 @@ class FiberSensorSuite extends QueryTest with SharedOapContext val fiberInfo2 = SparkListenerCustomInfoUpdate(host2, execId2, "OapFiberCacheHeartBeatMessager", CacheStatusSerDe .serialize(Seq(FiberCacheStatus(filePath, bitSet2, groupCount, fieldCount)))) - this.update(fiberInfo2) - assert(this.getHosts(filePath) contains (FiberSensor.OAP_CACHE_HOST_PREFIX + host2 + + fiberSensor.updateLocations(fiberInfo2) + assert(fiberSensor.getHosts(filePath) contains (FiberSensor.OAP_CACHE_HOST_PREFIX + host2 + FiberSensor.OAP_CACHE_EXECUTOR_PREFIX + execId2)) // executor3 update @@ -186,8 +195,8 @@ class FiberSensorSuite extends QueryTest with SharedOapContext val fiberInfo3 = SparkListenerCustomInfoUpdate(host3, execId3, "OapFiberCacheHeartBeatMessager", CacheStatusSerDe .serialize(Seq(FiberCacheStatus(filePath, bitSet3, groupCount, fieldCount)))) - this.update(fiberInfo3) - assert(this.getHosts(filePath) === Some(FiberSensor.OAP_CACHE_HOST_PREFIX + host2 + + fiberSensor.updateLocations(fiberInfo3) + assert(fiberSensor.getHosts(filePath) === Seq(FiberSensor.OAP_CACHE_HOST_PREFIX + host2 + FiberSensor.OAP_CACHE_EXECUTOR_PREFIX + execId2)) } } diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BTreeIndexScannerSuite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BTreeIndexScannerSuite.scala index b372e5bcfd2..79bc1d8dda2 100644 --- a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BTreeIndexScannerSuite.scala +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BTreeIndexScannerSuite.scala @@ -52,7 +52,7 @@ class BTreeIndexScannerSuite extends SharedOapContext { new SecurityManager(conf)))) } override def afterEach(): Unit = { - TaskContext.get().asInstanceOf[TaskContextImpl].markTaskCompleted() + // TaskContext.get().asInstanceOf[TaskContextImpl].markTaskCompleted() TaskContext.unset() } diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BTreeRecordReaderWriterV1Suite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BTreeRecordReaderWriterV1Suite.scala index fd9b3453de6..2696abf9e4c 100644 --- a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BTreeRecordReaderWriterV1Suite.scala +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BTreeRecordReaderWriterV1Suite.scala @@ -117,7 +117,7 @@ class BTreeRecordReaderWriterV1Suite extends SharedOapContext { nonNullKeyRecords.map(InternalRow(_)).foreach(writer.write(null, _)) nullKeyRecords.map(InternalRow(_)).foreach(writer.write(null, _)) writer.close(null) - TaskContext.get().asInstanceOf[TaskContextImpl].markTaskCompleted() + // TaskContext.get().asInstanceOf[TaskContextImpl].markTaskCompleted() TaskContext.unset() val indexData = fileWriter.toByteArray diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BTreeRecordReaderWriterV2Suite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BTreeRecordReaderWriterV2Suite.scala index c7807e8fa24..864b9e0dcd1 100644 --- a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BTreeRecordReaderWriterV2Suite.scala +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BTreeRecordReaderWriterV2Suite.scala @@ -114,7 +114,7 @@ class BTreeRecordReaderWriterV2Suite extends SharedOapContext { nonNullKeyRecords.map(InternalRow(_)).foreach(writer.write(null, _)) nullKeyRecords.map(InternalRow(_)).foreach(writer.write(null, _)) writer.close(null) - TaskContext.get().asInstanceOf[TaskContextImpl].markTaskCompleted() + // TaskContext.get().asInstanceOf[TaskContextImpl].markTaskCompleted() TaskContext.unset() val indexData = fileWriter.toByteArray diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BitmapAnalyzeStatisticsSuite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BitmapAnalyzeStatisticsSuite.scala index 74b8fbdf72f..79447c4749b 100644 --- a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BitmapAnalyzeStatisticsSuite.scala +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/BitmapAnalyzeStatisticsSuite.scala @@ -49,7 +49,7 @@ private[oap] class TestOapSessionWithRawLocalFileSystem(sc: SparkContext) trait SharedOapContextWithRawLocalFileSystem extends SharedOapContextBase { protected override def createSparkSession: TestSparkSession = { - new TestOapSessionWithRawLocalFileSystem(sparkConf) + new TestOapSessionWithRawLocalFileSystem(oapSparkConf) } } diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsIndexSuite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsIndexSuite.scala new file mode 100644 index 00000000000..227a3c4c8c2 --- /dev/null +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/index/elasticsearch/OapEsIndexSuite.scala @@ -0,0 +1,150 @@ +/* + * 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.datasources.oap.index.elasticsearch + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.test.oap.SharedOapContext +import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.util.Utils + +class OapEsIndexSuite extends SharedOapContext { + + + // Skip file leak checking + override def beforeEach(): Unit = {} + override def afterEach(): Unit = {} + + // scalastyle:off println + test("test create index ddl") { + val es = OapEsClientRepository.getOrCreateOapEsClient() + val path = Utils.createTempDir().getAbsolutePath + + if (es.checkEsIndexExists(OapEsProperties.ES_INDEX_NAME)) { + es.dropEsIndex(OapEsProperties.ES_INDEX_NAME) + } + es.createEsIndex(OapEsProperties.ES_INDEX_NAME) + + sql(s"""CREATE TEMPORARY VIEW oap_test (a INT, b STRING) + | USING oap + | OPTIONS (path '$path')""".stripMargin) + + val strings = Seq("abcde", "bbcdb", "aacbb", "baccc", "ddddd").toArray + val data = (1 to 100).map(i => (i, strings(i % strings.length))) + spark.createDataFrame(data).createOrReplaceTempView("t") + sql("INSERT OVERWRITE TABLE oap_test SELECT * FROM t") + + // sql("SELECT * FROM oap_test").show() + sql("CREATE OINDEX es_index ON oap_test (b) USING ES") + // sql("SELECT * FROM oap_test WHERE b = 'abcde'").show() + sql("SELECT * FROM oap_test WHERE b like '%ab%'").show() + + val file = new Path(path) + val l = file.getFileSystem(new Configuration()).listStatus(file) + + val indexFiles = l.filter(f => f.getPath.getName.contains("es_index")).map(_.getPath.getName) + println(indexFiles.mkString("\n")) + // es.batchDelete(indexFiles, "all_oap_es_index") + // println(es.queryAllRecordIds("all_oap_es_index").mkString(", ")) + + // println(l.map(f => f.getPath.getName).mkString("\n")) + + /* + es.batchDelete(indexFiles, "all_oap_es_index") + + es.dropESTableIndex("all_oap_es_index") + sql("DROP OINDEX es_index ON oap_test") + sqlContext.dropTempTable("oap_test") + */ + } + // scalastyle:on println + + /* + test("index exists check") { + es.createEsTableIndex(TEST_TABLE_NAME) + assert(es.checkEsTableIndexExists(TEST_TABLE_NAME)) + es.dropESTableIndex(TEST_TABLE_NAME) + assert(!es.checkEsTableIndexExists(TEST_TABLE_NAME)) + } + + ignore("es table index create") { + es.createEsTableIndex(TEST_TABLE_NAME) + assert(es.checkEsTableIndexExists(TEST_TABLE_NAME)) + es.dropESTableIndex(TEST_TABLE_NAME) + } + + ignore("es table index drop") { + es.createEsTableIndex(TEST_TABLE_NAME) + es.dropESTableIndex(TEST_TABLE_NAME) + assert(!es.checkEsTableIndexExists(TEST_TABLE_NAME)) + } + + ignore("insert records to es table") { + es.createEsTableIndex(TEST_TABLE_NAME) + val records = (1 to 10).map { x => + val key = InternalRow.fromSeq(UTF8String.fromString(s"test$x") :: Nil) + (key, x) + } + es.batchInsert(records, "data.idx1.index", TEST_TABLE_NAME) + val ids = es.queryAllRecordIds(TEST_TABLE_NAME) + assert(ids.toSet == (1 to 10).map(_.toString).toSet) + es.dropESTableIndex(TEST_TABLE_NAME) + } + + ignore("query records ids by index file name") { + es.createEsTableIndex(TEST_TABLE_NAME) + val records1 = (1 to 10).map { x => + val key = InternalRow.fromSeq(UTF8String.fromString(s"test$x") :: Nil) + (key, x) + } + es.batchInsert(records1, "data.idx1.index", TEST_TABLE_NAME) + + val records2 = (11 to 20).map { x => + val key = InternalRow.fromSeq(UTF8String.fromString(s"test$x") :: Nil) + (key, x) + } + es.batchInsert(records2, "data.idx2.index", TEST_TABLE_NAME) + + val ids = es.queryIdsByIndexFileName("data.idx1.index", TEST_TABLE_NAME) + assert(ids.toSet == (1 to 10).map(_.toString).toSet) + es.dropESTableIndex(TEST_TABLE_NAME) + } + + ignore("drop records for oap index from es") { + es.createEsTableIndex(TEST_TABLE_NAME) + val records1 = (1 to 10).map { x => + val key = InternalRow.fromSeq(UTF8String.fromString(s"test$x") :: Nil) + (key, x) + } + es.batchInsert(records1, "data.idx1.index", TEST_TABLE_NAME) + + val records2 = (11 to 20).map { x => + val key = InternalRow.fromSeq(UTF8String.fromString(s"test$x") :: Nil) + (key, x) + } + es.batchInsert(records2, "data.idx2.index", TEST_TABLE_NAME) + + es.batchDelete("data.idx1.index", TEST_TABLE_NAME) + + val ids = es.queryAllRecordIds(TEST_TABLE_NAME) + assert(ids.toSet == (11 to 20).map(_.toString).toSet) + es.dropESTableIndex(TEST_TABLE_NAME) + } + */ +} diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/io/ParquetDataFileSuite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/io/ParquetDataFileSuite.scala index 7c61f0c9e05..fb096094bde 100644 --- a/src/test/scala/org/apache/spark/sql/execution/datasources/oap/io/ParquetDataFileSuite.scala +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/oap/io/ParquetDataFileSuite.scala @@ -71,6 +71,8 @@ abstract class ParquetDataFileSuite extends SparkFunSuite with SharedOapContext SQLConf.PARQUET_INT96_AS_TIMESTAMP.defaultValue.get) configuration.setBoolean(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key, SQLConf.PARQUET_WRITE_LEGACY_FORMAT.defaultValue.get) + // SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS is defined in Spark 2.2 and later + configuration.setBoolean("spark.sql.parquet.int64AsTimestampMillis", false) prepareData() } @@ -78,6 +80,7 @@ abstract class ParquetDataFileSuite extends SparkFunSuite with SharedOapContext configuration.unset(SQLConf.PARQUET_BINARY_AS_STRING.key) configuration.unset(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key) configuration.unset(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key) + configuration.unset("spark.sql.parquet.int64AsTimestampMillis") cleanDir() } diff --git a/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index 5a0394e0210..bf3dfc4c464 100644 --- a/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -368,88 +368,6 @@ class ParquetSchemaSuite extends ParquetSchemaTest { } } - test("merge with metastore schema") { - // Field type conflict resolution - assertResult( - StructType(Seq( - StructField("lowerCase", StringType), - StructField("UPPERCase", DoubleType, nullable = false)))) { - - ParquetFileFormat.mergeMetastoreParquetSchema( - StructType(Seq( - StructField("lowercase", StringType), - StructField("uppercase", DoubleType, nullable = false))), - - StructType(Seq( - StructField("lowerCase", BinaryType), - StructField("UPPERCase", IntegerType, nullable = true)))) - } - - // MetaStore schema is subset of parquet schema - assertResult( - StructType(Seq( - StructField("UPPERCase", DoubleType, nullable = false)))) { - - ParquetFileFormat.mergeMetastoreParquetSchema( - StructType(Seq( - StructField("uppercase", DoubleType, nullable = false))), - - StructType(Seq( - StructField("lowerCase", BinaryType), - StructField("UPPERCase", IntegerType, nullable = true)))) - } - - // Metastore schema contains additional non-nullable fields. - assert(intercept[Throwable] { - ParquetFileFormat.mergeMetastoreParquetSchema( - StructType(Seq( - StructField("uppercase", DoubleType, nullable = false), - StructField("lowerCase", BinaryType, nullable = false))), - - StructType(Seq( - StructField("UPPERCase", IntegerType, nullable = true)))) - }.getMessage.contains("detected conflicting schemas")) - - // Conflicting non-nullable field names - intercept[Throwable] { - ParquetFileFormat.mergeMetastoreParquetSchema( - StructType(Seq(StructField("lower", StringType, nullable = false))), - StructType(Seq(StructField("lowerCase", BinaryType)))) - } - } - - test("merge missing nullable fields from Metastore schema") { - // Standard case: Metastore schema contains additional nullable fields not present - // in the Parquet file schema. - assertResult( - StructType(Seq( - StructField("firstField", StringType, nullable = true), - StructField("secondField", StringType, nullable = true), - StructField("thirdfield", StringType, nullable = true)))) { - ParquetFileFormat.mergeMetastoreParquetSchema( - StructType(Seq( - StructField("firstfield", StringType, nullable = true), - StructField("secondfield", StringType, nullable = true), - StructField("thirdfield", StringType, nullable = true))), - StructType(Seq( - StructField("firstField", StringType, nullable = true), - StructField("secondField", StringType, nullable = true)))) - } - - // Merge should fail if the Metastore contains any additional fields that are not - // nullable. - assert(intercept[Throwable] { - ParquetFileFormat.mergeMetastoreParquetSchema( - StructType(Seq( - StructField("firstfield", StringType, nullable = true), - StructField("secondfield", StringType, nullable = true), - StructField("thirdfield", StringType, nullable = false))), - StructType(Seq( - StructField("firstField", StringType, nullable = true), - StructField("secondField", StringType, nullable = true)))) - }.getMessage.contains("detected conflicting schemas")) - } - test("schema merging failure error message") { import testImplicits._ diff --git a/src/test/scala/org/apache/spark/sql/oap/rpc/OapRpcManagerSuite.scala b/src/test/scala/org/apache/spark/sql/oap/rpc/OapRpcManagerSuite.scala index e2b8302e8a7..15a69c33569 100644 --- a/src/test/scala/org/apache/spark/sql/oap/rpc/OapRpcManagerSuite.scala +++ b/src/test/scala/org/apache/spark/sql/oap/rpc/OapRpcManagerSuite.scala @@ -24,6 +24,7 @@ import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} import org.apache.spark._ import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv} import org.apache.spark.sql.internal.oap.OapConf +import org.apache.spark.sql.oap.adapter.RpcEndpointRefAdapter import org.apache.spark.sql.oap.rpc.OapMessages.{DummyHeartbeat, DummyMessage, Heartbeat, RegisterOapRpcManager} @@ -112,7 +113,8 @@ class OapRpcManagerSuite extends SparkFunSuite with BeforeAndAfterEach with Priv val rpcManagerSlaveEndpoint = spy(new OapRpcManagerSlaveEndpoint(rpcEnv, null)) val slaveEndpoint = rpcEnv.setupEndpoint( s"OapRpcManagerSlave_$executorId", rpcManagerSlaveEndpoint) - rpcDriverEndpoint.askWithRetry[Boolean](RegisterOapRpcManager(executorId, slaveEndpoint)) + RpcEndpointRefAdapter.askSync[Boolean]( + rpcDriverEndpoint, RegisterOapRpcManager(executorId, slaveEndpoint)) rpcManagerSlaveEndpoint } diff --git a/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala deleted file mode 100644 index cecfd990986..00000000000 --- a/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ /dev/null @@ -1,172 +0,0 @@ -/* - * 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.sources - -import java.text.NumberFormat -import java.util.Locale - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.hadoop.io.{NullWritable, Text} -import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} -import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat - -import org.apache.spark.sql.{sources, Row, SparkSession} -import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{Cast, Expression, GenericInternalRow, InterpretedPredicate, InterpretedProjection, JoinedRow, Literal} -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection -import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.types.{DataType, StructType} -import org.apache.spark.util.SerializableConfiguration - -class SimpleTextSource extends TextBasedFileFormat with DataSourceRegister { - override def shortName(): String = "test" - - override def inferSchema( - sparkSession: SparkSession, - options: Map[String, String], - files: Seq[FileStatus]): Option[StructType] = { - Some(DataType.fromJson(options("dataSchema")).asInstanceOf[StructType]) - } - - override def prepareWrite( - sparkSession: SparkSession, - job: Job, - options: Map[String, String], - dataSchema: StructType): OutputWriterFactory = { - SimpleTextRelation.lastHadoopConf = Option(job.getConfiguration) - new OutputWriterFactory { - override def newInstance( - path: String, - dataSchema: StructType, - context: TaskAttemptContext): OutputWriter = { - new SimpleTextOutputWriter(path, context) - } - - override def getFileExtension(context: TaskAttemptContext): String = "" - } - } - - override def buildReader( - sparkSession: SparkSession, - dataSchema: StructType, - partitionSchema: StructType, - requiredSchema: StructType, - filters: Seq[Filter], - options: Map[String, String], - hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { - SimpleTextRelation.lastHadoopConf = Option(hadoopConf) - SimpleTextRelation.requiredColumns = requiredSchema.fieldNames - SimpleTextRelation.pushedFilters = filters.toSet - - val fieldTypes = dataSchema.map(_.dataType) - val inputAttributes = dataSchema.toAttributes - val outputAttributes = requiredSchema.flatMap { field => - inputAttributes.find(_.name == field.name) - } - - val broadcastedHadoopConf = - sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) - - (file: PartitionedFile) => { - val predicate = { - val filterCondition: Expression = filters.collect { - // According to `unhandledFilters`, `SimpleTextRelation` only handles `GreaterThan` filter - case sources.GreaterThan(column, value) => - val dataType = dataSchema(column).dataType - val literal = Literal.create(value, dataType) - val attribute = inputAttributes.find(_.name == column).get - expressions.GreaterThan(attribute, literal) - }.reduceOption(expressions.And).getOrElse(Literal(true)) - InterpretedPredicate.create(filterCondition, inputAttributes) - } - - // Uses a simple projection to simulate column pruning - val projection = new InterpretedProjection(outputAttributes, inputAttributes) - - val unsafeRowIterator = - new HadoopFileLinesReader(file, broadcastedHadoopConf.value.value).map { line => - val record = line.toString - new GenericInternalRow(record.split(",", -1).zip(fieldTypes).map { - case (v, dataType) => - val value = if (v == "") null else v - // `Cast`ed values are always of internal types (e.g. UTF8String instead of String) - Cast(Literal(value), dataType).eval() - }) - }.filter(predicate).map(projection) - - // Appends partition values - val fullOutput = requiredSchema.toAttributes ++ partitionSchema.toAttributes - val joinedRow = new JoinedRow() - val appendPartitionColumns = GenerateUnsafeProjection.generate(fullOutput, fullOutput) - - unsafeRowIterator.map { dataRow => - appendPartitionColumns(joinedRow(dataRow, file.partitionValues)) - } - } - } -} - -class SimpleTextOutputWriter(path: String, context: TaskAttemptContext) - extends OutputWriter { - - private val recordWriter: RecordWriter[NullWritable, Text] = - new AppendingTextOutputFormat(path).getRecordWriter(context) - - override def write(row: Row): Unit = { - val serialized = row.toSeq.map { v => - if (v == null) "" else v.toString - }.mkString(",") - recordWriter.write(null, new Text(serialized)) - } - - override def close(): Unit = { - recordWriter.close(context) - } -} - -class AppendingTextOutputFormat(path: String) extends TextOutputFormat[NullWritable, Text] { - - val numberFormat = NumberFormat.getInstance(Locale.US) - numberFormat.setMinimumIntegerDigits(5) - numberFormat.setGroupingUsed(false) - - override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - new Path(path) - } -} - -object SimpleTextRelation { - // Used to test column pruning - var requiredColumns: Seq[String] = Nil - - // Used to test filter push-down - var pushedFilters: Set[Filter] = Set.empty - - // Used to test failed committer - var failCommitter = false - - // Used to test failed writer - var failWriter = false - - // Used to test failure callback - var callbackCalled = false - - // Used by the test case to check the value propagated in the hadoop confs. - var lastHadoopConf: Option[Configuration] = None -} diff --git a/src/test/scala/org/apache/spark/sql/test/oap/SharedOapContext.scala b/src/test/scala/org/apache/spark/sql/test/oap/SharedOapContext.scala index 1ace8f57fcd..aef2176a3dd 100644 --- a/src/test/scala/org/apache/spark/sql/test/oap/SharedOapContext.scala +++ b/src/test/scala/org/apache/spark/sql/test/oap/SharedOapContext.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.test.{SharedSQLContext, TestOapLocalClusterSession, trait SharedOapContext extends SharedOapContextBase { protected override def createSparkSession: TestSparkSession = { - new TestOapSession(sparkConf) + new TestOapSession(oapSparkConf) } } @@ -39,14 +39,17 @@ trait SharedOapContext extends SharedOapContextBase { */ trait SharedOapLocalClusterContext extends SharedOapContextBase { protected override def createSparkSession: TestSparkSession = { - new TestOapLocalClusterSession(sparkConf) + new TestOapLocalClusterSession(oapSparkConf) } } trait SharedOapContextBase extends SharedSQLContext { + // In Spark 2.1, sparkConf is a val. However is Spark 2.2 sparkConf is a function that create + // a new SparkConf each time. + val oapSparkConf = sparkConf // avoid the overflow of offHeap memory - sparkConf.set("spark.memory.offHeap.size", "100m") + oapSparkConf.set("spark.memory.offHeap.size", "100m") protected override def beforeAll(): Unit = { super.beforeAll() @@ -54,12 +57,10 @@ trait SharedOapContextBase extends SharedSQLContext { } // disable file based cbo for all test suite, as it always fails. - sparkConf.set(OapConf.OAP_EXECUTOR_INDEX_SELECTION_FILE_POLICY.key, "false") + oapSparkConf.set(OapConf.OAP_EXECUTOR_INDEX_SELECTION_FILE_POLICY.key, "false") protected lazy val configuration: Configuration = spark.sessionState.newHadoopConf() - protected implicit def sqlConf: SQLConf = sqlContext.conf - protected def getOapFileFormat(sparkPlan: SparkPlan): Set[Option[OapFileFormat]] = { def getOapFileFormatFromSource(node: SparkPlan): Option[OapFileFormat] = { node match {