From c47d0795fdd45943ab658828e21bf39bfaf99935 Mon Sep 17 00:00:00 2001 From: Saikat Kanjilal Date: Sun, 30 Oct 2016 11:43:47 -0700 Subject: [PATCH 001/132] initial set of changes for local[4] in core --- .../scala/org/apache/spark/ContextCleanerSuite.scala | 4 ++-- .../org/apache/spark/HeartbeatReceiverSuite.scala | 2 +- .../org/apache/spark/JobCancellationSuite.scala | 12 ++++++------ .../test/scala/org/apache/spark/SparkConfSuite.scala | 4 ++-- .../org/apache/spark/rdd/AsyncRDDActionsSuite.scala | 2 +- .../org/apache/spark/rdd/LocalCheckpointSuite.scala | 2 +- 6 files changed, 13 insertions(+), 13 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index f98150536d8a..adf5e9931635 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -44,7 +44,7 @@ abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[Ha { implicit val defaultTimeout = timeout(10000 millis) val conf = new SparkConf() - .setMaster("local[2]") + .setMaster("local[4]") .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") @@ -233,7 +233,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { // Verify that checkpoints are NOT cleaned up if the config is not enabled sc.stop() val conf = new SparkConf() - .setMaster("local[2]") + .setMaster("local[4]") .setAppName("cleanupCheckpoint") .set("spark.cleaner.referenceTracking.cleanCheckpoints", "false") sc = new SparkContext(conf) diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 3777d77f8f5b..8c2afe414729 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -68,7 +68,7 @@ class HeartbeatReceiverSuite override def beforeEach(): Unit = { super.beforeEach() val conf = new SparkConf() - .setMaster("local[2]") + .setMaster("local[4]") .setAppName("test") .set("spark.dynamicAllocation.testing", "true") sc = spy(new SparkContext(conf)) diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index c347ab8dc802..38a577e716de 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -47,7 +47,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft test("local mode, FIFO scheduler") { val conf = new SparkConf().set("spark.scheduler.mode", "FIFO") - sc = new SparkContext("local[2]", "test", conf) + sc = new SparkContext("local[4]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. @@ -58,7 +58,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft val conf = new SparkConf().set("spark.scheduler.mode", "FAIR") val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() conf.set("spark.scheduler.allocation.file", xmlPath) - sc = new SparkContext("local[2]", "test", conf) + sc = new SparkContext("local[4]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. @@ -115,7 +115,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft } test("job group") { - sc = new SparkContext("local[2]", "test") + sc = new SparkContext("local[4]", "test") // Add a listener to release the semaphore once any tasks are launched. val sem = new Semaphore(0) @@ -145,7 +145,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft } test("inherited job group (SPARK-6629)") { - sc = new SparkContext("local[2]", "test") + sc = new SparkContext("local[4]", "test") // Add a listener to release the semaphore once any tasks are launched. val sem = new Semaphore(0) @@ -180,7 +180,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft } test("job group with interruption") { - sc = new SparkContext("local[2]", "test") + sc = new SparkContext("local[4]", "test") // Add a listener to release the semaphore once any tasks are launched. val sem = new Semaphore(0) @@ -215,7 +215,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft // make sure the first stage is not finished until cancel is issued val sem1 = new Semaphore(0) - sc = new SparkContext("local[2]", "test") + sc = new SparkContext("local[4]", "test") sc.addSparkListener(new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { sem1.release() diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index a883d1b57e52..efbdb95dd503 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -129,8 +129,8 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst test("SparkContext property overriding") { val conf = new SparkConf(false).setMaster("local").setAppName("My app") - sc = new SparkContext("local[2]", "My other app", conf) - assert(sc.master === "local[2]") + sc = new SparkContext("local[4]", "My other app", conf) + assert(sc.master === "local[4]") assert(sc.appName === "My other app") } diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index d18bde790b40..37f60affca7f 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -35,7 +35,7 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim override def beforeAll() { super.beforeAll() - sc = new SparkContext("local[2]", "test") + sc = new SparkContext("local[4]", "test") } override def afterAll() { diff --git a/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala b/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala index 2802cd975292..5ff61b35c8bc 100644 --- a/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala @@ -28,7 +28,7 @@ class LocalCheckpointSuite extends SparkFunSuite with LocalSparkContext { override def beforeEach(): Unit = { super.beforeEach() - sc = new SparkContext("local[2]", "test") + sc = new SparkContext("local[4]", "test") } test("transform storage level") { From 90d3b91f4cb59d84fea7105d54ef8c87a7d5c6a2 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Sun, 30 Oct 2016 13:14:45 -0700 Subject: [PATCH 002/132] [SPARK-18103][SQL] Rename *FileCatalog to *FileIndex ## What changes were proposed in this pull request? To reduce the number of components in SQL named *Catalog, rename *FileCatalog to *FileIndex. A FileIndex is responsible for returning the list of partitions / files to scan given a filtering expression. ``` TableFileCatalog => CatalogFileIndex FileCatalog => FileIndex ListingFileCatalog => InMemoryFileIndex MetadataLogFileCatalog => MetadataLogFileIndex PrunedTableFileCatalog => PrunedInMemoryFileIndex ``` cc yhuai marmbrus ## How was this patch tested? N/A Author: Eric Liang Author: Eric Liang Closes #15634 from ericl/rename-file-provider. --- .../spark/metrics/source/StaticSources.scala | 2 +- .../spark/sql/execution/CacheManager.scala | 2 +- ...leCatalog.scala => CatalogFileIndex.scala} | 24 ++++++------- .../execution/datasources/DataSource.scala | 10 +++--- .../{FileCatalog.scala => FileIndex.scala} | 2 +- .../datasources/HadoopFsRelation.scala | 4 +-- ...eCatalog.scala => InMemoryFileIndex.scala} | 8 ++--- ...scala => PartitioningAwareFileIndex.scala} | 16 ++++----- .../PruneFileSourcePartitions.scala | 6 ++-- .../streaming/CompactibleFileStreamLog.scala | 4 +-- .../streaming/FileStreamSource.scala | 4 +-- .../streaming/MetadataLogFileCatalog.scala | 6 ++-- .../datasources/FileCatalogSuite.scala | 36 +++++++++---------- .../datasources/FileSourceStrategySuite.scala | 2 +- .../ParquetPartitionDiscoverySuite.scala | 2 +- .../sql/streaming/FileStreamSinkSuite.scala | 6 ++-- .../sql/streaming/FileStreamSourceSuite.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 4 +-- .../spark/sql/hive/CachedTableSuite.scala | 10 +++--- .../hive/PartitionedTablePerfStatsSuite.scala | 2 +- .../PruneFileSourcePartitionsSuite.scala | 6 ++-- 21 files changed, 79 insertions(+), 79 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/{TableFileCatalog.scala => CatalogFileIndex.scala} (83%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/{FileCatalog.scala => FileIndex.scala} (99%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/{ListingFileCatalog.scala => InMemoryFileIndex.scala} (92%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/{PartitioningAwareFileCatalog.scala => PartitioningAwareFileIndex.scala} (96%) diff --git a/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala b/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala index b54885b7ff8b..3f7cfd9d2c11 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/StaticSources.scala @@ -76,7 +76,7 @@ object HiveCatalogMetrics extends Source { val METRIC_PARTITIONS_FETCHED = metricRegistry.counter(MetricRegistry.name("partitionsFetched")) /** - * Tracks the total number of files discovered off of the filesystem by ListingFileCatalog. + * Tracks the total number of files discovered off of the filesystem by InMemoryFileIndex. */ val METRIC_FILES_DISCOVERED = metricRegistry.counter(MetricRegistry.name("filesDiscovered")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index fb72c679e362..526623a36d2a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -177,7 +177,7 @@ class CacheManager extends Logging { /** * Traverses a given `plan` and searches for the occurrences of `qualifiedPath` in the - * [[org.apache.spark.sql.execution.datasources.FileCatalog]] of any [[HadoopFsRelation]] nodes + * [[org.apache.spark.sql.execution.datasources.FileIndex]] of any [[HadoopFsRelation]] nodes * in the plan. If found, we refresh the metadata and return true. Otherwise, this method returns * false. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala similarity index 83% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala index b459df5734d4..092aabc89a36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala @@ -26,23 +26,23 @@ import org.apache.spark.sql.types.StructType /** - * A [[FileCatalog]] for a metastore catalog table. + * A [[FileIndex]] for a metastore catalog table. * * @param sparkSession a [[SparkSession]] * @param table the metadata of the table * @param sizeInBytes the table's data size in bytes */ -class TableFileCatalog( +class CatalogFileIndex( sparkSession: SparkSession, val table: CatalogTable, - override val sizeInBytes: Long) extends FileCatalog { + override val sizeInBytes: Long) extends FileIndex { protected val hadoopConf = sparkSession.sessionState.newHadoopConf private val fileStatusCache = FileStatusCache.newCache(sparkSession) assert(table.identifier.database.isDefined, - "The table identifier must be qualified in TableFileCatalog") + "The table identifier must be qualified in CatalogFileIndex") private val baseLocation = table.storage.locationUri @@ -57,12 +57,12 @@ class TableFileCatalog( override def refresh(): Unit = fileStatusCache.invalidateAll() /** - * Returns a [[ListingFileCatalog]] for this table restricted to the subset of partitions + * Returns a [[InMemoryFileIndex]] for this table restricted to the subset of partitions * specified by the given partition-pruning filters. * * @param filters partition-pruning filters */ - def filterPartitions(filters: Seq[Expression]): ListingFileCatalog = { + def filterPartitions(filters: Seq[Expression]): InMemoryFileIndex = { if (table.partitionColumnNames.nonEmpty) { val selectedPartitions = sparkSession.sessionState.catalog.listPartitionsByFilter( table.identifier, filters) @@ -70,20 +70,20 @@ class TableFileCatalog( PartitionPath(p.toRow(partitionSchema), p.storage.locationUri.get) } val partitionSpec = PartitionSpec(partitionSchema, partitions) - new PrunedTableFileCatalog( + new PrunedInMemoryFileIndex( sparkSession, new Path(baseLocation.get), fileStatusCache, partitionSpec) } else { - new ListingFileCatalog(sparkSession, rootPaths, table.storage.properties, None) + new InMemoryFileIndex(sparkSession, rootPaths, table.storage.properties, None) } } override def inputFiles: Array[String] = filterPartitions(Nil).inputFiles - // `TableFileCatalog` may be a member of `HadoopFsRelation`, `HadoopFsRelation` may be a member + // `CatalogFileIndex` may be a member of `HadoopFsRelation`, `HadoopFsRelation` may be a member // of `LogicalRelation`, and `LogicalRelation` may be used as the cache key. So we need to // implement `equals` and `hashCode` here, to make it work with cache lookup. override def equals(o: Any): Boolean = o match { - case other: TableFileCatalog => this.table.identifier == other.table.identifier + case other: CatalogFileIndex => this.table.identifier == other.table.identifier case _ => false } @@ -97,12 +97,12 @@ class TableFileCatalog( * @param tableBasePath The default base path of the Hive metastore table * @param partitionSpec The partition specifications from Hive metastore */ -private class PrunedTableFileCatalog( +private class PrunedInMemoryFileIndex( sparkSession: SparkSession, tableBasePath: Path, fileStatusCache: FileStatusCache, override val partitionSpec: PartitionSpec) - extends ListingFileCatalog( + extends InMemoryFileIndex( sparkSession, partitionSpec.partitions.map(_.path), Map.empty, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 5b8f05a39624..996109865fdc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -202,7 +202,7 @@ case class DataSource( val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) SparkHadoopUtil.get.globPathIfNecessary(qualified) }.toArray - val fileCatalog = new ListingFileCatalog(sparkSession, globbedPaths, options, None) + val fileCatalog = new InMemoryFileIndex(sparkSession, globbedPaths, options, None) val partitionSchema = fileCatalog.partitionSpec().partitionColumns val inferred = format.inferSchema( sparkSession, @@ -364,7 +364,7 @@ case class DataSource( case (format: FileFormat, _) if hasMetadata(caseInsensitiveOptions.get("path").toSeq ++ paths) => val basePath = new Path((caseInsensitiveOptions.get("path").toSeq ++ paths).head) - val fileCatalog = new MetadataLogFileCatalog(sparkSession, basePath) + val fileCatalog = new MetadataLogFileIndex(sparkSession, basePath) val dataSchema = userSpecifiedSchema.orElse { format.inferSchema( sparkSession, @@ -417,12 +417,12 @@ case class DataSource( val fileCatalog = if (sparkSession.sqlContext.conf.manageFilesourcePartitions && catalogTable.isDefined && catalogTable.get.partitionProviderIsHive) { - new TableFileCatalog( + new CatalogFileIndex( sparkSession, catalogTable.get, catalogTable.get.stats.map(_.sizeInBytes.toLong).getOrElse(0L)) } else { - new ListingFileCatalog( + new InMemoryFileIndex( sparkSession, globbedPaths, options, partitionSchema) } @@ -433,7 +433,7 @@ case class DataSource( format.inferSchema( sparkSession, caseInsensitiveOptions, - fileCatalog.asInstanceOf[ListingFileCatalog].allFiles()) + fileCatalog.asInstanceOf[InMemoryFileIndex].allFiles()) }.getOrElse { throw new AnalysisException( s"Unable to infer schema for $format at ${allPaths.take(2).mkString(",")}. " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileIndex.scala similarity index 99% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileIndex.scala index dba64624c34b..277223d52ec5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileIndex.scala @@ -33,7 +33,7 @@ case class PartitionDirectory(values: InternalRow, files: Seq[FileStatus]) * An interface for objects capable of enumerating the root paths of a relation as well as the * partitions of a relation subject to some pruning expressions. */ -trait FileCatalog { +trait FileIndex { /** * Returns the list of root input paths from which the catalog will get files. There may be a diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala index afad8898089b..014abd454f5c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types.StructType * Acts as a container for all of the metadata required to read from a datasource. All discovery, * resolution and merging logic for schemas and partitions has been removed. * - * @param location A [[FileCatalog]] that can enumerate the locations of all the files that + * @param location A [[FileIndex]] that can enumerate the locations of all the files that * comprise this relation. * @param partitionSchema The schema of the columns (if any) that are used to partition the relation * @param dataSchema The schema of any remaining columns. Note that if any partition columns are @@ -38,7 +38,7 @@ import org.apache.spark.sql.types.StructType * @param options Configuration used when reading / writing data. */ case class HadoopFsRelation( - location: FileCatalog, + location: FileIndex, partitionSchema: StructType, dataSchema: StructType, bucketSpec: Option[BucketSpec], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala similarity index 92% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala index d9d588388aaf..7531f0ae02e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ListingFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.types.StructType /** - * A [[FileCatalog]] that generates the list of files to process by recursively listing all the + * A [[FileIndex]] that generates the list of files to process by recursively listing all the * files present in `paths`. * * @param rootPaths the list of root table paths to scan @@ -34,13 +34,13 @@ import org.apache.spark.sql.types.StructType * @param partitionSchema an optional partition schema that will be use to provide types for the * discovered partitions */ -class ListingFileCatalog( +class InMemoryFileIndex( sparkSession: SparkSession, override val rootPaths: Seq[Path], parameters: Map[String, String], partitionSchema: Option[StructType], fileStatusCache: FileStatusCache = NoopCache) - extends PartitioningAwareFileCatalog( + extends PartitioningAwareFileIndex( sparkSession, parameters, partitionSchema, fileStatusCache) { @volatile private var cachedLeafFiles: mutable.LinkedHashMap[Path, FileStatus] = _ @@ -79,7 +79,7 @@ class ListingFileCatalog( } override def equals(other: Any): Boolean = other match { - case hdfs: ListingFileCatalog => rootPaths.toSet == hdfs.rootPaths.toSet + case hdfs: InMemoryFileIndex => rootPaths.toSet == hdfs.rootPaths.toSet case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala similarity index 96% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala index cc4049e92590..a8a722dd3c62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala @@ -34,19 +34,19 @@ import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.util.SerializableConfiguration /** - * An abstract class that represents [[FileCatalog]]s that are aware of partitioned tables. + * An abstract class that represents [[FileIndex]]s that are aware of partitioned tables. * It provides the necessary methods to parse partition data based on a set of files. * * @param parameters as set of options to control partition discovery * @param userPartitionSchema an optional partition schema that will be use to provide types for * the discovered partitions */ -abstract class PartitioningAwareFileCatalog( +abstract class PartitioningAwareFileIndex( sparkSession: SparkSession, parameters: Map[String, String], userPartitionSchema: Option[StructType], - fileStatusCache: FileStatusCache = NoopCache) extends FileCatalog with Logging { - import PartitioningAwareFileCatalog.BASE_PATH_PARAM + fileStatusCache: FileStatusCache = NoopCache) extends FileIndex with Logging { + import PartitioningAwareFileIndex.BASE_PATH_PARAM /** Returns the specification of the partitions inferred from the data. */ def partitionSpec(): PartitionSpec @@ -253,9 +253,9 @@ abstract class PartitioningAwareFileCatalog( } val discovered = if (pathsToFetch.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { - PartitioningAwareFileCatalog.listLeafFilesInParallel(pathsToFetch, hadoopConf, sparkSession) + PartitioningAwareFileIndex.listLeafFilesInParallel(pathsToFetch, hadoopConf, sparkSession) } else { - PartitioningAwareFileCatalog.listLeafFilesInSerial(pathsToFetch, hadoopConf) + PartitioningAwareFileIndex.listLeafFilesInSerial(pathsToFetch, hadoopConf) } discovered.foreach { case (path, leafFiles) => HiveCatalogMetrics.incrementFilesDiscovered(leafFiles.size) @@ -266,7 +266,7 @@ abstract class PartitioningAwareFileCatalog( } } -object PartitioningAwareFileCatalog extends Logging { +object PartitioningAwareFileIndex extends Logging { val BASE_PATH_PARAM = "basePath" /** A serializable variant of HDFS's BlockLocation. */ @@ -383,7 +383,7 @@ object PartitioningAwareFileCatalog extends Logging { if (shouldFilterOut(name)) { Seq.empty[FileStatus] } else { - // [SPARK-17599] Prevent ListingFileCatalog from failing if path doesn't exist + // [SPARK-17599] Prevent InMemoryFileIndex from failing if path doesn't exist // Note that statuses only include FileStatus for the files and dirs directly under path, // and does not include anything else recursively. val statuses = try fs.listStatus(path) catch { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 8689017c3ed7..8566a8061034 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -28,7 +28,7 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { logicalRelation @ LogicalRelation(fsRelation @ HadoopFsRelation( - tableFileCatalog: TableFileCatalog, + catalogFileIndex: CatalogFileIndex, partitionSchema, _, _, @@ -56,9 +56,9 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { ExpressionSet(normalizedFilters.filter(_.references.subsetOf(partitionSet))) if (partitionKeyFilters.nonEmpty) { - val prunedFileCatalog = tableFileCatalog.filterPartitions(partitionKeyFilters.toSeq) + val prunedFileIndex = catalogFileIndex.filterPartitions(partitionKeyFilters.toSeq) val prunedFsRelation = - fsRelation.copy(location = prunedFileCatalog)(sparkSession) + fsRelation.copy(location = prunedFileIndex)(sparkSession) val prunedLogicalRelation = logicalRelation.copy( relation = prunedFsRelation, expectedOutputAttributes = Some(logicalRelation.output)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala index c14feea91ed7..b26edeeb0400 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala @@ -146,7 +146,7 @@ abstract class CompactibleFileStreamLog[T: ClassTag]( */ def allFiles(): Array[T] = { var latestId = getLatest().map(_._1).getOrElse(-1L) - // There is a race condition when `FileStreamSink` is deleting old files and `StreamFileCatalog` + // There is a race condition when `FileStreamSink` is deleting old files and `StreamFileIndex` // is calling this method. This loop will retry the reading to deal with the // race condition. while (true) { @@ -158,7 +158,7 @@ abstract class CompactibleFileStreamLog[T: ClassTag]( } catch { case e: IOException => // Another process using `CompactibleFileStreamLog` may delete the batch files when - // `StreamFileCatalog` are reading. However, it only happens when a compaction is + // `StreamFileIndex` are reading. However, it only happens when a compaction is // deleting old files. If so, let's try the next compaction batch and we should find it. // Otherwise, this is a real IO issue and we should throw it. latestId = nextCompactionBatchId(latestId, compactInterval) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index a392b8299902..680df01acc1a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} -import org.apache.spark.sql.execution.datasources.{DataSource, ListingFileCatalog, LogicalRelation} +import org.apache.spark.sql.execution.datasources.{DataSource, InMemoryFileIndex, LogicalRelation} import org.apache.spark.sql.types.StructType /** @@ -156,7 +156,7 @@ class FileStreamSource( private def fetchAllFiles(): Seq[(String, Long)] = { val startTime = System.nanoTime val globbedPaths = SparkHadoopUtil.get.globPathIfNecessary(qualifiedBasePath) - val catalog = new ListingFileCatalog(sparkSession, globbedPaths, options, Some(new StructType)) + val catalog = new InMemoryFileIndex(sparkSession, globbedPaths, options, Some(new StructType)) val files = catalog.allFiles().sortBy(_.getModificationTime).map { status => (status.getPath.toUri.toString, status.getModificationTime) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala index 82b67cb1ca6e..aeaa13473693 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala @@ -26,11 +26,11 @@ import org.apache.spark.sql.execution.datasources._ /** - * A [[FileCatalog]] that generates the list of files to processing by reading them from the + * A [[FileIndex]] that generates the list of files to processing by reading them from the * metadata log files generated by the [[FileStreamSink]]. */ -class MetadataLogFileCatalog(sparkSession: SparkSession, path: Path) - extends PartitioningAwareFileCatalog(sparkSession, Map.empty, None) { +class MetadataLogFileIndex(sparkSession: SparkSession, path: Path) + extends PartitioningAwareFileIndex(sparkSession, Map.empty, None) { private val metadataDirectory = new Path(path, FileStreamSink.metadataDir) logInfo(s"Reading streaming file log from $metadataDirectory") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala index 9c43169cbf89..56df1face636 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileCatalogSuite.scala @@ -28,15 +28,15 @@ import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.test.SharedSQLContext -class FileCatalogSuite extends SharedSQLContext { +class FileIndexSuite extends SharedSQLContext { - test("ListingFileCatalog: leaf files are qualified paths") { + test("InMemoryFileIndex: leaf files are qualified paths") { withTempDir { dir => val file = new File(dir, "text.txt") stringToFile(file, "text") val path = new Path(file.getCanonicalPath) - val catalog = new ListingFileCatalog(spark, Seq(path), Map.empty, None) { + val catalog = new InMemoryFileIndex(spark, Seq(path), Map.empty, None) { def leafFilePaths: Seq[Path] = leafFiles.keys.toSeq def leafDirPaths: Seq[Path] = leafDirToChildrenFiles.keys.toSeq } @@ -45,7 +45,7 @@ class FileCatalogSuite extends SharedSQLContext { } } - test("ListingFileCatalog: input paths are converted to qualified paths") { + test("InMemoryFileIndex: input paths are converted to qualified paths") { withTempDir { dir => val file = new File(dir, "text.txt") stringToFile(file, "text") @@ -59,42 +59,42 @@ class FileCatalogSuite extends SharedSQLContext { val qualifiedFilePath = fs.makeQualified(new Path(file.getCanonicalPath)) require(qualifiedFilePath.toString.startsWith("file:")) - val catalog1 = new ListingFileCatalog( + val catalog1 = new InMemoryFileIndex( spark, Seq(unqualifiedDirPath), Map.empty, None) assert(catalog1.allFiles.map(_.getPath) === Seq(qualifiedFilePath)) - val catalog2 = new ListingFileCatalog( + val catalog2 = new InMemoryFileIndex( spark, Seq(unqualifiedFilePath), Map.empty, None) assert(catalog2.allFiles.map(_.getPath) === Seq(qualifiedFilePath)) } } - test("ListingFileCatalog: folders that don't exist don't throw exceptions") { + test("InMemoryFileIndex: folders that don't exist don't throw exceptions") { withTempDir { dir => val deletedFolder = new File(dir, "deleted") assert(!deletedFolder.exists()) - val catalog1 = new ListingFileCatalog( + val catalog1 = new InMemoryFileIndex( spark, Seq(new Path(deletedFolder.getCanonicalPath)), Map.empty, None) // doesn't throw an exception assert(catalog1.listLeafFiles(catalog1.rootPaths).isEmpty) } } - test("PartitioningAwareFileCatalog - file filtering") { - assert(!PartitioningAwareFileCatalog.shouldFilterOut("abcd")) - assert(PartitioningAwareFileCatalog.shouldFilterOut(".ab")) - assert(PartitioningAwareFileCatalog.shouldFilterOut("_cd")) - assert(!PartitioningAwareFileCatalog.shouldFilterOut("_metadata")) - assert(!PartitioningAwareFileCatalog.shouldFilterOut("_common_metadata")) - assert(PartitioningAwareFileCatalog.shouldFilterOut("_ab_metadata")) - assert(PartitioningAwareFileCatalog.shouldFilterOut("_cd_common_metadata")) + test("PartitioningAwareFileIndex - file filtering") { + assert(!PartitioningAwareFileIndex.shouldFilterOut("abcd")) + assert(PartitioningAwareFileIndex.shouldFilterOut(".ab")) + assert(PartitioningAwareFileIndex.shouldFilterOut("_cd")) + assert(!PartitioningAwareFileIndex.shouldFilterOut("_metadata")) + assert(!PartitioningAwareFileIndex.shouldFilterOut("_common_metadata")) + assert(PartitioningAwareFileIndex.shouldFilterOut("_ab_metadata")) + assert(PartitioningAwareFileIndex.shouldFilterOut("_cd_common_metadata")) } - test("SPARK-17613 - PartitioningAwareFileCatalog: base path w/o '/' at end") { + test("SPARK-17613 - PartitioningAwareFileIndex: base path w/o '/' at end") { class MockCatalog( override val rootPaths: Seq[Path]) - extends PartitioningAwareFileCatalog(spark, Map.empty, None) { + extends PartitioningAwareFileIndex(spark, Map.empty, None) { override def refresh(): Unit = {} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index c32254d9dfde..d900ce7bb237 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -393,7 +393,7 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi util.stringToFile(file, fileName) } - val fileCatalog = new ListingFileCatalog( + val fileCatalog = new InMemoryFileIndex( sparkSession = spark, rootPaths = Seq(new Path(tempDir)), parameters = Map.empty[String, String], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index f2a209e91962..120a3a2ef33a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -634,7 +634,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha val queryExecution = spark.read.parquet(dir.getCanonicalPath).queryExecution queryExecution.analyzed.collectFirst { case LogicalRelation( - HadoopFsRelation(location: PartitioningAwareFileCatalog, _, _, _, _, _), _, _) => + HadoopFsRelation(location: PartitioningAwareFileIndex, _, _, _, _, _), _, _) => assert(location.partitionSpec() === PartitionSpec.emptySpec) }.getOrElse { fail(s"Expecting a matching HadoopFsRelation, but got:\n$queryExecution") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 19c89f5c4100..18b42a81a098 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.streaming.{FileStreamSinkWriter, MemoryStream, MetadataLogFileCatalog} +import org.apache.spark.sql.execution.streaming.{FileStreamSinkWriter, MemoryStream, MetadataLogFileIndex} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StructField, StructType} @@ -179,14 +179,14 @@ class FileStreamSinkSuite extends StreamTest { .add(StructField("id", IntegerType)) assert(outputDf.schema === expectedSchema) - // Verify that MetadataLogFileCatalog is being used and the correct partitioning schema has + // Verify that MetadataLogFileIndex is being used and the correct partitioning schema has // been inferred val hadoopdFsRelations = outputDf.queryExecution.analyzed.collect { case LogicalRelation(baseRelation, _, _) if baseRelation.isInstanceOf[HadoopFsRelation] => baseRelation.asInstanceOf[HadoopFsRelation] } assert(hadoopdFsRelations.size === 1) - assert(hadoopdFsRelations.head.location.isInstanceOf[MetadataLogFileCatalog]) + assert(hadoopdFsRelations.head.location.isInstanceOf[MetadataLogFileIndex]) assert(hadoopdFsRelations.head.partitionSchema.exists(_.name == "id")) assert(hadoopdFsRelations.head.dataSchema.exists(_.name == "value")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index b9e9da9a1ec5..47018b3a3c49 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -879,7 +879,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { val numFiles = 10000 // This is to avoid running a spark job to list of files in parallel - // by the ListingFileCatalog. + // by the InMemoryFileIndex. spark.sessionState.conf.setConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD, numFiles * 2) withTempDirs { case (root, tmp) => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index d1de863ce362..624ab747e442 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -200,7 +200,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val rootPaths: Seq[Path] = if (lazyPruningEnabled) { Seq(metastoreRelation.hiveQlTable.getDataLocation) } else { - // By convention (for example, see TableFileCatalog), the definition of a + // By convention (for example, see CatalogFileIndex), the definition of a // partitioned table's paths depends on whether that table has any actual partitions. // Partitioned tables without partitions use the location of the table's base path. // Partitioned tables with partitions use the locations of those partitions' data @@ -227,7 +227,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val logicalRelation = cached.getOrElse { val sizeInBytes = metastoreRelation.statistics.sizeInBytes.toLong val fileCatalog = { - val catalog = new TableFileCatalog( + val catalog = new CatalogFileIndex( sparkSession, metastoreRelation.catalogTable, sizeInBytes) if (lazyPruningEnabled) { catalog diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index ecdf4f14b398..fc35304c80ec 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.{AnalysisException, Dataset, QueryTest, SaveMode} import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, TableFileCatalog} +import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils @@ -321,17 +321,17 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto sql("DROP TABLE cachedTable") } - test("cache a table using TableFileCatalog") { + test("cache a table using CatalogFileIndex") { withTable("test") { sql("CREATE TABLE test(i int) PARTITIONED BY (p int) STORED AS parquet") val tableMeta = spark.sharedState.externalCatalog.getTable("default", "test") - val tableFileCatalog = new TableFileCatalog(spark, tableMeta, 0) + val catalogFileIndex = new CatalogFileIndex(spark, tableMeta, 0) val dataSchema = StructType(tableMeta.schema.filterNot { f => tableMeta.partitionColumnNames.contains(f.name) }) val relation = HadoopFsRelation( - location = tableFileCatalog, + location = catalogFileIndex, partitionSchema = tableMeta.partitionSchema, dataSchema = dataSchema, bucketSpec = None, @@ -343,7 +343,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto assert(spark.sharedState.cacheManager.lookupCachedData(plan).isDefined) - val sameCatalog = new TableFileCatalog(spark, tableMeta, 0) + val sameCatalog = new CatalogFileIndex(spark, tableMeta, 0) val sameRelation = HadoopFsRelation( location = sameCatalog, partitionSchema = tableMeta.partitionSchema, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala index 476383a5b33a..d8e31c4e39a5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionedTablePerfStatsSuite.scala @@ -256,7 +256,7 @@ class PartitionedTablePerfStatsSuite // of doing plan cache validation based on the entire partition set. HiveCatalogMetrics.reset() assert(spark.sql("select * from test where partCol1 = 999").count() == 0) - // 5 from table resolution, another 5 from ListingFileCatalog + // 5 from table resolution, another 5 from InMemoryFileIndex assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 10) assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala index 59639aacf3a3..cdbc26cd5c57 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PruneFileSourcePartitions, TableFileCatalog} +import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, LogicalRelation, PruneFileSourcePartitions} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils @@ -45,13 +45,13 @@ class PruneFileSourcePartitionsSuite extends QueryTest with SQLTestUtils with Te |LOCATION '${dir.getAbsolutePath}'""".stripMargin) val tableMeta = spark.sharedState.externalCatalog.getTable("default", "test") - val tableFileCatalog = new TableFileCatalog(spark, tableMeta, 0) + val catalogFileIndex = new CatalogFileIndex(spark, tableMeta, 0) val dataSchema = StructType(tableMeta.schema.filterNot { f => tableMeta.partitionColumnNames.contains(f.name) }) val relation = HadoopFsRelation( - location = tableFileCatalog, + location = catalogFileIndex, partitionSchema = tableMeta.partitionSchema, dataSchema = dataSchema, bucketSpec = None, From 522359bd808c8c866db0b0e5c70ce0796426a854 Mon Sep 17 00:00:00 2001 From: Saikat Kanjilal Date: Sun, 30 Oct 2016 15:06:47 -0700 Subject: [PATCH 003/132] added mllib changes to local[4] --- .../org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java | 2 +- .../classification/JavaStreamingLogisticRegressionSuite.java | 2 +- .../apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java | 2 +- .../mllib/regression/JavaStreamingLinearRegressionSuite.java | 2 +- .../java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java | 2 +- .../org/apache/spark/mllib/util/MLlibTestSparkContext.scala | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java index 01ff1ea65861..683f53e18f22 100644 --- a/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/util/JavaDefaultReadWriteSuite.java @@ -37,7 +37,7 @@ public class JavaDefaultReadWriteSuite { @Before public void setUp() { - jsc = new JavaSparkContext("local[2]", "JavaDefaultReadWriteSuite"); + jsc = new JavaSparkContext("local[4]", "JavaDefaultReadWriteSuite"); SQLContext.clearActive(); sqlContext = new SQLContext(jsc); SQLContext.setActive(sqlContext); diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java index c9e5ee22f327..989bd326d87a 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaStreamingLogisticRegressionSuite.java @@ -44,7 +44,7 @@ public class JavaStreamingLogisticRegressionSuite implements Serializable { @Before public void setUp() { SparkConf conf = new SparkConf() - .setMaster("local[2]") + .setMaster("local[4]") .setAppName("test") .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); ssc = new JavaStreamingContext(conf, new Duration(1000)); diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java index d644766d1e54..a154d01f259d 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaStreamingKMeansSuite.java @@ -44,7 +44,7 @@ public class JavaStreamingKMeansSuite implements Serializable { @Before public void setUp() { SparkConf conf = new SparkConf() - .setMaster("local[2]") + .setMaster("local[4]") .setAppName("test") .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); ssc = new JavaStreamingContext(conf, new Duration(1000)); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java index dbf6488d4108..00baf4998c66 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaStreamingLinearRegressionSuite.java @@ -43,7 +43,7 @@ public class JavaStreamingLinearRegressionSuite implements Serializable { @Before public void setUp() { SparkConf conf = new SparkConf() - .setMaster("local[2]") + .setMaster("local[4]") .setAppName("test") .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); ssc = new JavaStreamingContext(conf, new Duration(1000)); diff --git a/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java index 66b2ceacb05f..ab73359e9ef1 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/stat/JavaStatisticsSuite.java @@ -49,7 +49,7 @@ public class JavaStatisticsSuite implements Serializable { @Before public void setUp() { SparkConf conf = new SparkConf() - .setMaster("local[2]") + .setMaster("local[4]") .setAppName("JavaStatistics") .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); sc = new JavaSparkContext(conf); diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala index ebcd591465cb..9cbcb9649e3f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLlibTestSparkContext.scala @@ -29,7 +29,7 @@ trait MLlibTestSparkContext extends BeforeAndAfterAll { self: Suite => override def beforeAll() { super.beforeAll() val conf = new SparkConf() - .setMaster("local[2]") + .setMaster("local[4]") .setAppName("MLlibUnitTest") sc = new SparkContext(conf) SQLContext.clearActive() From 8ae2da0b2551011e2f6cf02907a1e20c138a4b2f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 30 Oct 2016 23:24:30 +0100 Subject: [PATCH 004/132] [SPARK-18106][SQL] ANALYZE TABLE should raise a ParseException for invalid option ## What changes were proposed in this pull request? Currently, `ANALYZE TABLE` command accepts `identifier` for option `NOSCAN`. This PR raises a ParseException for unknown option. **Before** ```scala scala> sql("create table test(a int)") res0: org.apache.spark.sql.DataFrame = [] scala> sql("analyze table test compute statistics blah") res1: org.apache.spark.sql.DataFrame = [] ``` **After** ```scala scala> sql("create table test(a int)") res0: org.apache.spark.sql.DataFrame = [] scala> sql("analyze table test compute statistics blah") org.apache.spark.sql.catalyst.parser.ParseException: Expected `NOSCAN` instead of `blah`(line 1, pos 0) ``` ## How was this patch tested? Pass the Jenkins test with a new test case. Author: Dongjoon Hyun Closes #15640 from dongjoon-hyun/SPARK-18106. --- .../spark/sql/execution/SparkSqlParser.scala | 10 +++++++--- .../sql/execution/SparkSqlParserSuite.scala | 18 ++++++++++++++++-- 2 files changed, 23 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 1cc166d5a7a9..fe183d0097d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -98,9 +98,13 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitAnalyze(ctx: AnalyzeContext): LogicalPlan = withOrigin(ctx) { - if (ctx.partitionSpec == null && - ctx.identifier != null && - ctx.identifier.getText.toLowerCase == "noscan") { + if (ctx.partitionSpec != null) { + logWarning(s"Partition specification is ignored: ${ctx.partitionSpec.getText}") + } + if (ctx.identifier != null) { + if (ctx.identifier.getText.toLowerCase != "noscan") { + throw new ParseException(s"Expected `NOSCAN` instead of `${ctx.identifier.getText}`", ctx) + } AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier)) } else if (ctx.identifierSeq() == null) { AnalyzeTableCommand(visitTableIdentifier(ctx.tableIdentifier), noscan = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 679150e9ae4c..797fe9ffa8be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -23,8 +23,8 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.command.{DescribeFunctionCommand, DescribeTableCommand, - ShowFunctionsCommand} +import org.apache.spark.sql.execution.command.{AnalyzeTableCommand, DescribeFunctionCommand, + DescribeTableCommand, ShowFunctionsCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, CreateTempViewUsing} import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} @@ -220,4 +220,18 @@ class SparkSqlParserSuite extends PlanTest { intercept("explain describe tables x", "Unsupported SQL statement") } + + test("SPARK-18106 analyze table") { + assertEqual("analyze table t compute statistics", + AnalyzeTableCommand(TableIdentifier("t"), noscan = false)) + assertEqual("analyze table t compute statistics noscan", + AnalyzeTableCommand(TableIdentifier("t"), noscan = true)) + assertEqual("analyze table t partition (a) compute statistics noscan", + AnalyzeTableCommand(TableIdentifier("t"), noscan = true)) + + intercept("analyze table t compute statistics xxxx", + "Expected `NOSCAN` instead of `xxxx`") + intercept("analyze table t partition (a) compute statistics xxxx", + "Expected `NOSCAN` instead of `xxxx`") + } } From 2881a2d1d1a650a91df2c6a01275eba14a43b42a Mon Sep 17 00:00:00 2001 From: Hossein Date: Sun, 30 Oct 2016 16:17:23 -0700 Subject: [PATCH 005/132] [SPARK-17919] Make timeout to RBackend configurable in SparkR ## What changes were proposed in this pull request? This patch makes RBackend connection timeout configurable by user. ## How was this patch tested? N/A Author: Hossein Closes #15471 from falaki/SPARK-17919. --- R/pkg/R/backend.R | 20 ++++++++-- R/pkg/R/client.R | 2 +- R/pkg/R/sparkR.R | 8 +++- R/pkg/inst/worker/daemon.R | 4 +- R/pkg/inst/worker/worker.R | 7 +++- .../org/apache/spark/api/r/RBackend.scala | 15 ++++++- .../apache/spark/api/r/RBackendHandler.scala | 39 +++++++++++++++++-- .../org/apache/spark/api/r/RRunner.scala | 3 ++ .../apache/spark/api/r/SparkRDefaults.scala | 30 ++++++++++++++ .../org/apache/spark/deploy/RRunner.scala | 7 +++- docs/configuration.md | 15 +++++++ 11 files changed, 134 insertions(+), 16 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/api/r/SparkRDefaults.scala diff --git a/R/pkg/R/backend.R b/R/pkg/R/backend.R index 03e70bb2cb82..0a789e6c379d 100644 --- a/R/pkg/R/backend.R +++ b/R/pkg/R/backend.R @@ -108,13 +108,27 @@ invokeJava <- function(isStatic, objId, methodName, ...) { conn <- get(".sparkRCon", .sparkREnv) writeBin(requestMessage, conn) - # TODO: check the status code to output error information returnStatus <- readInt(conn) + handleErrors(returnStatus, conn) + + # Backend will send +1 as keep alive value to prevent various connection timeouts + # on very long running jobs. See spark.r.heartBeatInterval + while (returnStatus == 1) { + returnStatus <- readInt(conn) + handleErrors(returnStatus, conn) + } + + readObject(conn) +} + +# Helper function to check for returned errors and print appropriate error message to user +handleErrors <- function(returnStatus, conn) { if (length(returnStatus) == 0) { stop("No status is returned. Java SparkR backend might have failed.") } - if (returnStatus != 0) { + + # 0 is success and +1 is reserved for heartbeats. Other negative values indicate errors. + if (returnStatus < 0) { stop(readString(conn)) } - readObject(conn) } diff --git a/R/pkg/R/client.R b/R/pkg/R/client.R index 2d341d836c13..9d82814211bc 100644 --- a/R/pkg/R/client.R +++ b/R/pkg/R/client.R @@ -19,7 +19,7 @@ # Creates a SparkR client connection object # if one doesn't already exist -connectBackend <- function(hostname, port, timeout = 6000) { +connectBackend <- function(hostname, port, timeout) { if (exists(".sparkRcon", envir = .sparkREnv)) { if (isOpen(.sparkREnv[[".sparkRCon"]])) { cat("SparkRBackend client connection already exists\n") diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index cc6d591bb2f4..6b4a2f2fdc85 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -154,6 +154,7 @@ sparkR.sparkContext <- function( packages <- processSparkPackages(sparkPackages) existingPort <- Sys.getenv("EXISTING_SPARKR_BACKEND_PORT", "") + connectionTimeout <- as.numeric(Sys.getenv("SPARKR_BACKEND_CONNECTION_TIMEOUT", "6000")) if (existingPort != "") { if (length(packages) != 0) { warning(paste("sparkPackages has no effect when using spark-submit or sparkR shell", @@ -187,6 +188,7 @@ sparkR.sparkContext <- function( backendPort <- readInt(f) monitorPort <- readInt(f) rLibPath <- readString(f) + connectionTimeout <- readInt(f) close(f) file.remove(path) if (length(backendPort) == 0 || backendPort == 0 || @@ -194,7 +196,9 @@ sparkR.sparkContext <- function( length(rLibPath) != 1) { stop("JVM failed to launch") } - assign(".monitorConn", socketConnection(port = monitorPort), envir = .sparkREnv) + assign(".monitorConn", + socketConnection(port = monitorPort, timeout = connectionTimeout), + envir = .sparkREnv) assign(".backendLaunched", 1, envir = .sparkREnv) if (rLibPath != "") { assign(".libPath", rLibPath, envir = .sparkREnv) @@ -204,7 +208,7 @@ sparkR.sparkContext <- function( .sparkREnv$backendPort <- backendPort tryCatch({ - connectBackend("localhost", backendPort) + connectBackend("localhost", backendPort, timeout = connectionTimeout) }, error = function(err) { stop("Failed to connect JVM\n") diff --git a/R/pkg/inst/worker/daemon.R b/R/pkg/inst/worker/daemon.R index b92e6be995ca..3a318b71ea06 100644 --- a/R/pkg/inst/worker/daemon.R +++ b/R/pkg/inst/worker/daemon.R @@ -18,6 +18,7 @@ # Worker daemon rLibDir <- Sys.getenv("SPARKR_RLIBDIR") +connectionTimeout <- as.integer(Sys.getenv("SPARKR_BACKEND_CONNECTION_TIMEOUT", "6000")) dirs <- strsplit(rLibDir, ",")[[1]] script <- file.path(dirs[[1]], "SparkR", "worker", "worker.R") @@ -26,7 +27,8 @@ script <- file.path(dirs[[1]], "SparkR", "worker", "worker.R") suppressPackageStartupMessages(library(SparkR)) port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) -inputCon <- socketConnection(port = port, open = "rb", blocking = TRUE, timeout = 3600) +inputCon <- socketConnection( + port = port, open = "rb", blocking = TRUE, timeout = connectionTimeout) while (TRUE) { ready <- socketSelect(list(inputCon)) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index cfe41ded200c..03e745014786 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -90,6 +90,7 @@ bootTime <- currentTimeSecs() bootElap <- elapsedSecs() rLibDir <- Sys.getenv("SPARKR_RLIBDIR") +connectionTimeout <- as.integer(Sys.getenv("SPARKR_BACKEND_CONNECTION_TIMEOUT", "6000")) dirs <- strsplit(rLibDir, ",")[[1]] # Set libPaths to include SparkR package as loadNamespace needs this # TODO: Figure out if we can avoid this by not loading any objects that require @@ -98,8 +99,10 @@ dirs <- strsplit(rLibDir, ",")[[1]] suppressPackageStartupMessages(library(SparkR)) port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) -inputCon <- socketConnection(port = port, blocking = TRUE, open = "rb") -outputCon <- socketConnection(port = port, blocking = TRUE, open = "wb") +inputCon <- socketConnection( + port = port, blocking = TRUE, open = "rb", timeout = connectionTimeout) +outputCon <- socketConnection( + port = port, blocking = TRUE, open = "wb", timeout = connectionTimeout) # read the index of the current partition inside the RDD partition <- SparkR:::readInt(inputCon) diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala index 41d0a85ee3ad..550746c552d0 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala @@ -22,12 +22,13 @@ import java.net.{InetAddress, InetSocketAddress, ServerSocket} import java.util.concurrent.TimeUnit import io.netty.bootstrap.ServerBootstrap -import io.netty.channel.{ChannelFuture, ChannelInitializer, EventLoopGroup} +import io.netty.channel.{ChannelFuture, ChannelInitializer, ChannelOption, EventLoopGroup} import io.netty.channel.nio.NioEventLoopGroup import io.netty.channel.socket.SocketChannel import io.netty.channel.socket.nio.NioServerSocketChannel import io.netty.handler.codec.LengthFieldBasedFrameDecoder import io.netty.handler.codec.bytes.{ByteArrayDecoder, ByteArrayEncoder} +import io.netty.handler.timeout.ReadTimeoutHandler import org.apache.spark.SparkConf import org.apache.spark.internal.Logging @@ -43,7 +44,10 @@ private[spark] class RBackend { def init(): Int = { val conf = new SparkConf() - bossGroup = new NioEventLoopGroup(conf.getInt("spark.r.numRBackendThreads", 2)) + val backendConnectionTimeout = conf.getInt( + "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) + bossGroup = new NioEventLoopGroup( + conf.getInt("spark.r.numRBackendThreads", SparkRDefaults.DEFAULT_NUM_RBACKEND_THREADS)) val workerGroup = bossGroup val handler = new RBackendHandler(this) @@ -63,6 +67,7 @@ private[spark] class RBackend { // initialBytesToStrip = 4, i.e. strip out the length field itself new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4)) .addLast("decoder", new ByteArrayDecoder()) + .addLast("readTimeoutHandler", new ReadTimeoutHandler(backendConnectionTimeout)) .addLast("handler", handler) } }) @@ -110,6 +115,11 @@ private[spark] object RBackend extends Logging { val boundPort = sparkRBackend.init() val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost")) val listenPort = serverSocket.getLocalPort() + // Connection timeout is set by socket client. To make it configurable we will pass the + // timeout value to client inside the temp file + val conf = new SparkConf() + val backendConnectionTimeout = conf.getInt( + "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) // tell the R process via temporary file val path = args(0) @@ -118,6 +128,7 @@ private[spark] object RBackend extends Logging { dos.writeInt(boundPort) dos.writeInt(listenPort) SerDe.writeString(dos, RUtils.rPackages.getOrElse("")) + dos.writeInt(backendConnectionTimeout) dos.close() f.renameTo(new File(path)) diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala index 1422ef888fd4..9f5afa29d6d2 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -18,16 +18,19 @@ package org.apache.spark.api.r import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} +import java.util.concurrent.TimeUnit import scala.collection.mutable.HashMap import scala.language.existentials import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} import io.netty.channel.ChannelHandler.Sharable +import io.netty.handler.timeout.ReadTimeoutException import org.apache.spark.api.r.SerDe._ import org.apache.spark.internal.Logging -import org.apache.spark.util.Utils +import org.apache.spark.SparkConf +import org.apache.spark.util.{ThreadUtils, Utils} /** * Handler for RBackend @@ -83,7 +86,29 @@ private[r] class RBackendHandler(server: RBackend) writeString(dos, s"Error: unknown method $methodName") } } else { + // To avoid timeouts when reading results in SparkR driver, we will be regularly sending + // heartbeat responses. We use special code +1 to signal the client that backend is + // alive and it should continue blocking for result. + val execService = ThreadUtils.newDaemonSingleThreadScheduledExecutor("SparkRKeepAliveThread") + val pingRunner = new Runnable { + override def run(): Unit = { + val pingBaos = new ByteArrayOutputStream() + val pingDaos = new DataOutputStream(pingBaos) + writeInt(pingDaos, +1) + ctx.write(pingBaos.toByteArray) + } + } + val conf = new SparkConf() + val heartBeatInterval = conf.getInt( + "spark.r.heartBeatInterval", SparkRDefaults.DEFAULT_HEARTBEAT_INTERVAL) + val backendConnectionTimeout = conf.getInt( + "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) + val interval = Math.min(heartBeatInterval, backendConnectionTimeout - 1) + + execService.scheduleAtFixedRate(pingRunner, interval, interval, TimeUnit.SECONDS) handleMethodCall(isStatic, objId, methodName, numArgs, dis, dos) + execService.shutdown() + execService.awaitTermination(1, TimeUnit.SECONDS) } val reply = bos.toByteArray @@ -95,9 +120,15 @@ private[r] class RBackendHandler(server: RBackend) } override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { - // Close the connection when an exception is raised. - cause.printStackTrace() - ctx.close() + cause match { + case timeout: ReadTimeoutException => + // Do nothing. We don't want to timeout on read + logWarning("Ignoring read timeout in RBackendHandler") + case _ => + // Close the connection when an exception is raised. + cause.printStackTrace() + ctx.close() + } } def handleMethodCall( diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala index 496fdf851f7d..7ef64723d959 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -333,6 +333,8 @@ private[r] object RRunner { var rCommand = sparkConf.get("spark.sparkr.r.command", "Rscript") rCommand = sparkConf.get("spark.r.command", rCommand) + val rConnectionTimeout = sparkConf.getInt( + "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) val rOptions = "--vanilla" val rLibDir = RUtils.sparkRPackagePath(isDriver = false) val rExecScript = rLibDir(0) + "/SparkR/worker/" + script @@ -344,6 +346,7 @@ private[r] object RRunner { pb.environment().put("R_TESTS", "") pb.environment().put("SPARKR_RLIBDIR", rLibDir.mkString(",")) pb.environment().put("SPARKR_WORKER_PORT", port.toString) + pb.environment().put("SPARKR_BACKEND_CONNECTION_TIMEOUT", rConnectionTimeout.toString) pb.redirectErrorStream(true) // redirect stderr into stdout val proc = pb.start() val errThread = startStdoutThread(proc) diff --git a/core/src/main/scala/org/apache/spark/api/r/SparkRDefaults.scala b/core/src/main/scala/org/apache/spark/api/r/SparkRDefaults.scala new file mode 100644 index 000000000000..af67cbbce4e5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/SparkRDefaults.scala @@ -0,0 +1,30 @@ +/* + * 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.api.r + +private[spark] object SparkRDefaults { + + // Default value for spark.r.backendConnectionTimeout config + val DEFAULT_CONNECTION_TIMEOUT: Int = 6000 + + // Default value for spark.r.heartBeatInterval config + val DEFAULT_HEARTBEAT_INTERVAL: Int = 100 + + // Default value for spark.r.numRBackendThreads config + val DEFAULT_NUM_RBACKEND_THREADS = 2 +} diff --git a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala index d0466830b217..6eb53a825220 100644 --- a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.apache.spark.{SparkException, SparkUserAppException} -import org.apache.spark.api.r.{RBackend, RUtils} +import org.apache.spark.api.r.{RBackend, RUtils, SparkRDefaults} import org.apache.spark.util.RedirectThread /** @@ -51,6 +51,10 @@ object RRunner { cmd } + // Connection timeout set by R process on its connection to RBackend in seconds. + val backendConnectionTimeout = sys.props.getOrElse( + "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT.toString) + // Check if the file path exists. // If not, change directory to current working directory for YARN cluster mode val rF = new File(rFile) @@ -81,6 +85,7 @@ object RRunner { val builder = new ProcessBuilder((Seq(rCommand, rFileNormalized) ++ otherArgs).asJava) val env = builder.environment() env.put("EXISTING_SPARKR_BACKEND_PORT", sparkRBackendPort.toString) + env.put("SPARKR_BACKEND_CONNECTION_TIMEOUT", backendConnectionTimeout) val rPackageDir = RUtils.sparkRPackagePath(isDriver = true) // Put the R package directories into an env variable of comma-separated paths env.put("SPARKR_PACKAGE_DIR", rPackageDir.mkString(",")) diff --git a/docs/configuration.md b/docs/configuration.md index 6600cb6c0ac0..780fc94908d3 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1890,6 +1890,21 @@ showDF(properties, numRows = 200, truncate = FALSE) spark.r.shell.command is used for sparkR shell while spark.r.driver.command is used for running R script. + + spark.r.backendConnectionTimeout + 6000 + + Connection timeout set by R process on its connection to RBackend in seconds. + + + + spark.r.heartBeatInterval + 100 + + Interval for heartbeats sents from SparkR backend to R process to prevent connection timeout. + + + #### Deploy From b6879b8b3518c71c23262554fcb0fdad60287011 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Sun, 30 Oct 2016 16:19:19 -0700 Subject: [PATCH 006/132] [SPARK-16137][SPARKR] randomForest for R ## What changes were proposed in this pull request? Random Forest Regression and Classification for R Clean-up/reordering generics.R ## How was this patch tested? manual tests, unit tests Author: Felix Cheung Closes #15607 from felixcheung/rrandomforest. --- R/pkg/NAMESPACE | 9 +- R/pkg/R/generics.R | 66 ++--- R/pkg/R/mllib.R | 252 +++++++++++++++++- R/pkg/inst/tests/testthat/test_mllib.R | 68 +++++ .../org/apache/spark/ml/r/RWrappers.scala | 4 + .../r/RandomForestClassificationWrapper.scala | 147 ++++++++++ .../ml/r/RandomForestRegressionWrapper.scala | 144 ++++++++++ 7 files changed, 656 insertions(+), 34 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassificationWrapper.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressionWrapper.scala diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 7a89c01fee73..9cd6269f9a8f 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -44,7 +44,8 @@ exportMethods("glm", "spark.gaussianMixture", "spark.als", "spark.kstest", - "spark.logit") + "spark.logit", + "spark.randomForest") # Job group lifecycle management methods export("setJobGroup", @@ -350,7 +351,9 @@ export("as.DataFrame", "uncacheTable", "print.summary.GeneralizedLinearRegressionModel", "read.ml", - "print.summary.KSTest") + "print.summary.KSTest", + "print.summary.RandomForestRegressionModel", + "print.summary.RandomForestClassificationModel") export("structField", "structField.jobj", @@ -375,6 +378,8 @@ S3method(print, structField) S3method(print, structType) S3method(print, summary.GeneralizedLinearRegressionModel) S3method(print, summary.KSTest) +S3method(print, summary.RandomForestRegressionModel) +S3method(print, summary.RandomForestClassificationModel) S3method(structField, character) S3method(structField, jobj) S3method(structType, jobj) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 107e1c638be7..0271b26a10a9 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1310,9 +1310,11 @@ setGeneric("window", function(x, ...) { standardGeneric("window") }) #' @export setGeneric("year", function(x) { standardGeneric("year") }) -#' @rdname spark.glm +###################### Spark.ML Methods ########################## + +#' @rdname fitted #' @export -setGeneric("spark.glm", function(data, formula, ...) { standardGeneric("spark.glm") }) +setGeneric("fitted") #' @param x,y For \code{glm}: logical values indicating whether the response vector #' and model matrix used in the fitting process should be returned as @@ -1332,13 +1334,38 @@ setGeneric("predict", function(object, ...) { standardGeneric("predict") }) #' @export setGeneric("rbind", signature = "...") +#' @rdname spark.als +#' @export +setGeneric("spark.als", function(data, ...) { standardGeneric("spark.als") }) + +#' @rdname spark.gaussianMixture +#' @export +setGeneric("spark.gaussianMixture", + function(data, formula, ...) { standardGeneric("spark.gaussianMixture") }) + +#' @rdname spark.glm +#' @export +setGeneric("spark.glm", function(data, formula, ...) { standardGeneric("spark.glm") }) + +#' @rdname spark.isoreg +#' @export +setGeneric("spark.isoreg", function(data, formula, ...) { standardGeneric("spark.isoreg") }) + #' @rdname spark.kmeans #' @export setGeneric("spark.kmeans", function(data, formula, ...) { standardGeneric("spark.kmeans") }) -#' @rdname fitted +#' @rdname spark.kstest #' @export -setGeneric("fitted") +setGeneric("spark.kstest", function(data, ...) { standardGeneric("spark.kstest") }) + +#' @rdname spark.lda +#' @export +setGeneric("spark.lda", function(data, ...) { standardGeneric("spark.lda") }) + +#' @rdname spark.logit +#' @export +setGeneric("spark.logit", function(data, formula, ...) { standardGeneric("spark.logit") }) #' @rdname spark.mlp #' @export @@ -1348,13 +1375,14 @@ setGeneric("spark.mlp", function(data, ...) { standardGeneric("spark.mlp") }) #' @export setGeneric("spark.naiveBayes", function(data, formula, ...) { standardGeneric("spark.naiveBayes") }) -#' @rdname spark.survreg +#' @rdname spark.randomForest #' @export -setGeneric("spark.survreg", function(data, formula) { standardGeneric("spark.survreg") }) +setGeneric("spark.randomForest", + function(data, formula, ...) { standardGeneric("spark.randomForest") }) -#' @rdname spark.lda +#' @rdname spark.survreg #' @export -setGeneric("spark.lda", function(data, ...) { standardGeneric("spark.lda") }) +setGeneric("spark.survreg", function(data, formula) { standardGeneric("spark.survreg") }) #' @rdname spark.lda #' @export @@ -1364,20 +1392,6 @@ setGeneric("spark.posterior", function(object, newData) { standardGeneric("spark #' @export setGeneric("spark.perplexity", function(object, data) { standardGeneric("spark.perplexity") }) -#' @rdname spark.isoreg -#' @export -setGeneric("spark.isoreg", function(data, formula, ...) { standardGeneric("spark.isoreg") }) - -#' @rdname spark.gaussianMixture -#' @export -setGeneric("spark.gaussianMixture", - function(data, formula, ...) { - standardGeneric("spark.gaussianMixture") - }) - -#' @rdname spark.logit -#' @export -setGeneric("spark.logit", function(data, formula, ...) { standardGeneric("spark.logit") }) #' @param object a fitted ML model object. #' @param path the directory where the model is saved. @@ -1385,11 +1399,3 @@ setGeneric("spark.logit", function(data, formula, ...) { standardGeneric("spark. #' @rdname write.ml #' @export setGeneric("write.ml", function(object, path, ...) { standardGeneric("write.ml") }) - -#' @rdname spark.als -#' @export -setGeneric("spark.als", function(data, ...) { standardGeneric("spark.als") }) - -#' @rdname spark.kstest -#' @export -setGeneric("spark.kstest", function(data, ...) { standardGeneric("spark.kstest") }) diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index 629f284b79f3..7a220b8d53a2 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -102,6 +102,20 @@ setClass("KSTest", representation(jobj = "jobj")) #' @note LogisticRegressionModel since 2.1.0 setClass("LogisticRegressionModel", representation(jobj = "jobj")) +#' S4 class that represents a RandomForestRegressionModel +#' +#' @param jobj a Java object reference to the backing Scala RandomForestRegressionModel +#' @export +#' @note RandomForestRegressionModel since 2.1.0 +setClass("RandomForestRegressionModel", representation(jobj = "jobj")) + +#' S4 class that represents a RandomForestClassificationModel +#' +#' @param jobj a Java object reference to the backing Scala RandomForestClassificationModel +#' @export +#' @note RandomForestClassificationModel since 2.1.0 +setClass("RandomForestClassificationModel", representation(jobj = "jobj")) + #' Saves the MLlib model to the input path #' #' Saves the MLlib model to the input path. For more information, see the specific @@ -112,7 +126,7 @@ setClass("LogisticRegressionModel", representation(jobj = "jobj")) #' @seealso \link{spark.glm}, \link{glm}, #' @seealso \link{spark.als}, \link{spark.gaussianMixture}, \link{spark.isoreg}, \link{spark.kmeans}, #' @seealso \link{spark.lda}, \link{spark.logit}, \link{spark.mlp}, \link{spark.naiveBayes}, -#' @seealso \link{spark.survreg} +#' @seealso \link{spark.randomForest}, \link{spark.survreg}, #' @seealso \link{read.ml} NULL @@ -125,7 +139,8 @@ NULL #' @export #' @seealso \link{spark.glm}, \link{glm}, #' @seealso \link{spark.als}, \link{spark.gaussianMixture}, \link{spark.isoreg}, \link{spark.kmeans}, -#' @seealso \link{spark.logit}, \link{spark.mlp}, \link{spark.naiveBayes}, \link{spark.survreg} +#' @seealso \link{spark.logit}, \link{spark.mlp}, \link{spark.naiveBayes}, +#' @seealso \link{spark.randomForest}, \link{spark.survreg} NULL write_internal <- function(object, path, overwrite = FALSE) { @@ -1122,6 +1137,10 @@ read.ml <- function(path) { new("ALSModel", jobj = jobj) } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.LogisticRegressionWrapper")) { new("LogisticRegressionModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.RandomForestRegressorWrapper")) { + new("RandomForestRegressionModel", jobj = jobj) + } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.RandomForestClassifierWrapper")) { + new("RandomForestClassificationModel", jobj = jobj) } else { stop("Unsupported model: ", jobj) } @@ -1617,3 +1636,232 @@ print.summary.KSTest <- function(x, ...) { cat(summaryStr, "\n") invisible(x) } + +#' Random Forest Model for Regression and Classification +#' +#' \code{spark.randomForest} fits a Random Forest Regression model or Classification model on +#' a SparkDataFrame. Users can call \code{summary} to get a summary of the fitted Random Forest +#' model, \code{predict} to make predictions on new data, and \code{write.ml}/\code{read.ml} to +#' save/load fitted models. +#' For more details, see +#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html}{Random Forest} +#' +#' @param data a SparkDataFrame for training. +#' @param formula a symbolic description of the model to be fitted. Currently only a few formula +#' operators are supported, including '~', ':', '+', and '-'. +#' @param type type of model, one of "regression" or "classification", to fit +#' @param maxDepth Maximum depth of the tree (>= 0). (default = 5) +#' @param maxBins Maximum number of bins used for discretizing continuous features and for choosing +#' how to split on features at each node. More bins give higher granularity. Must be +#' >= 2 and >= number of categories in any categorical feature. (default = 32) +#' @param numTrees Number of trees to train (>= 1). +#' @param impurity Criterion used for information gain calculation. +#' For regression, must be "variance". For classification, must be one of +#' "entropy" and "gini". (default = gini) +#' @param minInstancesPerNode Minimum number of instances each child must have after split. +#' @param minInfoGain Minimum information gain for a split to be considered at a tree node. +#' @param checkpointInterval Param for set checkpoint interval (>= 1) or disable checkpoint (-1). +#' @param featureSubsetStrategy The number of features to consider for splits at each tree node. +#' Supported options: "auto", "all", "onethird", "sqrt", "log2", (0.0-1.0], [1-n]. +#' @param seed integer seed for random number generation. +#' @param subsamplingRate Fraction of the training data used for learning each decision tree, in +#' range (0, 1]. (default = 1.0) +#' @param probabilityCol column name for predicted class conditional probabilities, only for +#' classification. (default = "probability") +#' @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. +#' @param cacheNodeIds If FALSE, the algorithm will pass trees to executors to match instances with +#' nodes. +#' @param ... additional arguments passed to the method. +#' @aliases spark.randomForest,SparkDataFrame,formula-method +#' @return \code{spark.randomForest} returns a fitted Random Forest model. +#' @rdname spark.randomForest +#' @name spark.randomForest +#' @export +#' @examples +#' \dontrun{ +#' # fit a Random Forest Regression Model +#' df <- createDataFrame(longley) +#' model <- spark.randomForest(df, Employed ~ ., type = "regression", maxDepth = 5, maxBins = 16) +#' +#' # get the summary of the model +#' summary(model) +#' +#' # make predictions +#' predictions <- predict(model, df) +#' +#' # save and load the model +#' path <- "path/to/model" +#' write.ml(model, path) +#' savedModel <- read.ml(path) +#' summary(savedModel) +#' +#' # fit a Random Forest Classification Model +#' df <- createDataFrame(iris) +#' model <- spark.randomForest(df, Species ~ Petal_Length + Petal_Width, "classification") +#' } +#' @note spark.randomForest since 2.1.0 +setMethod("spark.randomForest", signature(data = "SparkDataFrame", formula = "formula"), + function(data, formula, type = c("regression", "classification"), + maxDepth = 5, maxBins = 32, numTrees = 20, impurity = NULL, + minInstancesPerNode = 1, minInfoGain = 0.0, checkpointInterval = 10, + featureSubsetStrategy = "auto", seed = NULL, subsamplingRate = 1.0, + probabilityCol = "probability", maxMemoryInMB = 256, cacheNodeIds = FALSE) { + type <- match.arg(type) + formula <- paste(deparse(formula), collapse = "") + if (!is.null(seed)) { + seed <- as.character(as.integer(seed)) + } + switch(type, + regression = { + if (is.null(impurity)) impurity <- "variance" + impurity <- match.arg(impurity, "variance") + jobj <- callJStatic("org.apache.spark.ml.r.RandomForestRegressorWrapper", + "fit", data@sdf, formula, as.integer(maxDepth), + as.integer(maxBins), as.integer(numTrees), + impurity, as.integer(minInstancesPerNode), + as.numeric(minInfoGain), as.integer(checkpointInterval), + as.character(featureSubsetStrategy), seed, + as.numeric(subsamplingRate), + as.integer(maxMemoryInMB), as.logical(cacheNodeIds)) + new("RandomForestRegressionModel", jobj = jobj) + }, + classification = { + if (is.null(impurity)) impurity <- "gini" + impurity <- match.arg(impurity, c("gini", "entropy")) + jobj <- callJStatic("org.apache.spark.ml.r.RandomForestClassifierWrapper", + "fit", data@sdf, formula, as.integer(maxDepth), + as.integer(maxBins), as.integer(numTrees), + impurity, as.integer(minInstancesPerNode), + as.numeric(minInfoGain), as.integer(checkpointInterval), + as.character(featureSubsetStrategy), seed, + as.numeric(subsamplingRate), as.character(probabilityCol), + as.integer(maxMemoryInMB), as.logical(cacheNodeIds)) + new("RandomForestClassificationModel", jobj = jobj) + } + ) + }) + +# Makes predictions from a Random Forest Regression model or Classification model + +#' @param newData a SparkDataFrame for testing. +#' @return \code{predict} returns a SparkDataFrame containing predicted labeled in a column named +#' "prediction" +#' @rdname spark.randomForest +#' @aliases predict,RandomForestRegressionModel-method +#' @export +#' @note predict(randomForestRegressionModel) since 2.1.0 +setMethod("predict", signature(object = "RandomForestRegressionModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +#' @rdname spark.randomForest +#' @aliases predict,RandomForestClassificationModel-method +#' @export +#' @note predict(randomForestClassificationModel) since 2.1.0 +setMethod("predict", signature(object = "RandomForestClassificationModel"), + function(object, newData) { + predict_internal(object, newData) + }) + +# Save the Random Forest Regression or Classification model to the input path. + +#' @param object A fitted Random Forest regression model or classification model +#' @param path The directory where the model is saved +#' @param overwrite Overwrites or not if the output path already exists. Default is FALSE +#' which means throw exception if the output path exists. +#' +#' @aliases write.ml,RandomForestRegressionModel,character-method +#' @rdname spark.randomForest +#' @export +#' @note write.ml(RandomForestRegressionModel, character) since 2.1.0 +setMethod("write.ml", signature(object = "RandomForestRegressionModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +#' @aliases write.ml,RandomForestClassificationModel,character-method +#' @rdname spark.randomForest +#' @export +#' @note write.ml(RandomForestClassificationModel, character) since 2.1.0 +setMethod("write.ml", signature(object = "RandomForestClassificationModel", path = "character"), + function(object, path, overwrite = FALSE) { + write_internal(object, path, overwrite) + }) + +# Get the summary of an RandomForestRegressionModel model +summary.randomForest <- function(model) { + jobj <- model@jobj + formula <- callJMethod(jobj, "formula") + numFeatures <- callJMethod(jobj, "numFeatures") + features <- callJMethod(jobj, "features") + featureImportances <- callJMethod(callJMethod(jobj, "featureImportances"), "toString") + numTrees <- callJMethod(jobj, "numTrees") + treeWeights <- callJMethod(jobj, "treeWeights") + list(formula = formula, + numFeatures = numFeatures, + features = features, + featureImportances = featureImportances, + numTrees = numTrees, + treeWeights = treeWeights, + jobj = jobj) +} + +#' @return \code{summary} returns the model's features as lists, depth and number of nodes +#' or number of classes. +#' @rdname spark.randomForest +#' @aliases summary,RandomForestRegressionModel-method +#' @export +#' @note summary(RandomForestRegressionModel) since 2.1.0 +setMethod("summary", signature(object = "RandomForestRegressionModel"), + function(object) { + ans <- summary.randomForest(object) + class(ans) <- "summary.RandomForestRegressionModel" + ans + }) + +# Get the summary of an RandomForestClassificationModel model + +#' @rdname spark.randomForest +#' @aliases summary,RandomForestClassificationModel-method +#' @export +#' @note summary(RandomForestClassificationModel) since 2.1.0 +setMethod("summary", signature(object = "RandomForestClassificationModel"), + function(object) { + ans <- summary.randomForest(object) + class(ans) <- "summary.RandomForestClassificationModel" + ans + }) + +# Prints the summary of Random Forest Regression Model +print.summary.randomForest <- function(x) { + jobj <- x$jobj + cat("Formula: ", x$formula) + cat("\nNumber of features: ", x$numFeatures) + cat("\nFeatures: ", unlist(x$features)) + cat("\nFeature importances: ", x$featureImportances) + cat("\nNumber of trees: ", x$numTrees) + cat("\nTree weights: ", unlist(x$treeWeights)) + + summaryStr <- callJMethod(jobj, "summary") + cat("\n", summaryStr, "\n") + invisible(x) +} + +#' @param x summary object of Random Forest regression model or classification model +#' returned by \code{summary}. +#' @rdname spark.randomForest +#' @export +#' @note print.summary.RandomForestRegressionModel since 2.1.0 +print.summary.RandomForestRegressionModel <- function(x, ...) { + print.summary.randomForest(x) +} + +# Prints the summary of Random Forest Classification Model + +#' @rdname spark.randomForest +#' @export +#' @note print.summary.RandomForestClassificationModel since 2.1.0 +print.summary.RandomForestClassificationModel <- function(x, ...) { + print.summary.randomForest(x) +} diff --git a/R/pkg/inst/tests/testthat/test_mllib.R b/R/pkg/inst/tests/testthat/test_mllib.R index 6d1fccc7c058..db98d0e45547 100644 --- a/R/pkg/inst/tests/testthat/test_mllib.R +++ b/R/pkg/inst/tests/testthat/test_mllib.R @@ -871,4 +871,72 @@ test_that("spark.kstest", { expect_match(capture.output(stats)[1], "Kolmogorov-Smirnov test summary:") }) +test_that("spark.randomForest Regression", { + data <- suppressWarnings(createDataFrame(longley)) + model <- spark.randomForest(data, Employed ~ ., "regression", maxDepth = 5, maxBins = 16, + numTrees = 1) + + predictions <- collect(predict(model, data)) + expect_equal(predictions$prediction, c(60.323, 61.122, 60.171, 61.187, + 63.221, 63.639, 64.989, 63.761, + 66.019, 67.857, 68.169, 66.513, + 68.655, 69.564, 69.331, 70.551), + tolerance = 1e-4) + + stats <- summary(model) + expect_equal(stats$numTrees, 1) + expect_error(capture.output(stats), NA) + expect_true(length(capture.output(stats)) > 6) + + model <- spark.randomForest(data, Employed ~ ., "regression", maxDepth = 5, maxBins = 16, + numTrees = 20, seed = 123) + predictions <- collect(predict(model, data)) + expect_equal(predictions$prediction, c(60.379, 61.096, 60.636, 62.258, + 63.736, 64.296, 64.868, 64.300, + 66.709, 67.697, 67.966, 67.252, + 68.866, 69.593, 69.195, 69.658), + tolerance = 1e-4) + stats <- summary(model) + expect_equal(stats$numTrees, 20) + + modelPath <- tempfile(pattern = "spark-randomForestRegression", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + stats2 <- summary(model2) + expect_equal(stats$formula, stats2$formula) + expect_equal(stats$numFeatures, stats2$numFeatures) + expect_equal(stats$features, stats2$features) + expect_equal(stats$featureImportances, stats2$featureImportances) + expect_equal(stats$numTrees, stats2$numTrees) + expect_equal(stats$treeWeights, stats2$treeWeights) + + unlink(modelPath) +}) + +test_that("spark.randomForest Classification", { + data <- suppressWarnings(createDataFrame(iris)) + model <- spark.randomForest(data, Species ~ Petal_Length + Petal_Width, "classification", + maxDepth = 5, maxBins = 16) + + stats <- summary(model) + expect_equal(stats$numFeatures, 2) + expect_equal(stats$numTrees, 20) + expect_error(capture.output(stats), NA) + expect_true(length(capture.output(stats)) > 6) + + modelPath <- tempfile(pattern = "spark-randomForestClassification", fileext = ".tmp") + write.ml(model, modelPath) + expect_error(write.ml(model, modelPath)) + write.ml(model, modelPath, overwrite = TRUE) + model2 <- read.ml(modelPath) + stats2 <- summary(model2) + expect_equal(stats$depth, stats2$depth) + expect_equal(stats$numNodes, stats2$numNodes) + expect_equal(stats$numClasses, stats2$numClasses) + + unlink(modelPath) +}) + sparkR.session.stop() diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala index 1df3662a5822..0e09e18027ca 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala @@ -56,6 +56,10 @@ private[r] object RWrappers extends MLReader[Object] { ALSWrapper.load(path) case "org.apache.spark.ml.r.LogisticRegressionWrapper" => LogisticRegressionWrapper.load(path) + case "org.apache.spark.ml.r.RandomForestRegressorWrapper" => + RandomForestRegressorWrapper.load(path) + case "org.apache.spark.ml.r.RandomForestClassifierWrapper" => + RandomForestClassifierWrapper.load(path) case _ => throw new SparkException(s"SparkR read.ml does not support load $className") } diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassificationWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassificationWrapper.scala new file mode 100644 index 000000000000..b0088ddaf3b1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassificationWrapper.scala @@ -0,0 +1,147 @@ +/* + * 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.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.classification.{RandomForestClassificationModel, RandomForestClassifier} +import org.apache.spark.ml.feature.RFormula +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class RandomForestClassifierWrapper private ( + val pipeline: PipelineModel, + val formula: String, + val features: Array[String]) extends MLWritable { + + private val DTModel: RandomForestClassificationModel = + pipeline.stages(1).asInstanceOf[RandomForestClassificationModel] + + lazy val numFeatures: Int = DTModel.numFeatures + lazy val featureImportances: Vector = DTModel.featureImportances + lazy val numTrees: Int = DTModel.getNumTrees + lazy val treeWeights: Array[Double] = DTModel.treeWeights + + def summary: String = DTModel.toDebugString + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset).drop(DTModel.getFeaturesCol) + } + + override def write: MLWriter = new + RandomForestClassifierWrapper.RandomForestClassifierWrapperWriter(this) +} + +private[r] object RandomForestClassifierWrapper extends MLReadable[RandomForestClassifierWrapper] { + def fit( // scalastyle:ignore + data: DataFrame, + formula: String, + maxDepth: Int, + maxBins: Int, + numTrees: Int, + impurity: String, + minInstancesPerNode: Int, + minInfoGain: Double, + checkpointInterval: Int, + featureSubsetStrategy: String, + seed: String, + subsamplingRate: Double, + probabilityCol: String, + maxMemoryInMB: Int, + cacheNodeIds: Boolean): RandomForestClassifierWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + RWrapperUtils.checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // get feature names from output schema + val schema = rFormulaModel.transform(data).schema + val featureAttrs = AttributeGroup.fromStructField(schema(rFormulaModel.getFeaturesCol)) + .attributes.get + val features = featureAttrs.map(_.name.get) + + // assemble and fit the pipeline + val rfc = new RandomForestClassifier() + .setMaxDepth(maxDepth) + .setMaxBins(maxBins) + .setNumTrees(numTrees) + .setImpurity(impurity) + .setMinInstancesPerNode(minInstancesPerNode) + .setMinInfoGain(minInfoGain) + .setCheckpointInterval(checkpointInterval) + .setFeatureSubsetStrategy(featureSubsetStrategy) + .setSubsamplingRate(subsamplingRate) + .setMaxMemoryInMB(maxMemoryInMB) + .setCacheNodeIds(cacheNodeIds) + .setProbabilityCol(probabilityCol) + .setFeaturesCol(rFormula.getFeaturesCol) + if (seed != null && seed.length > 0) rfc.setSeed(seed.toLong) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, rfc)) + .fit(data) + + new RandomForestClassifierWrapper(pipeline, formula, features) + } + + override def read: MLReader[RandomForestClassifierWrapper] = + new RandomForestClassifierWrapperReader + + override def load(path: String): RandomForestClassifierWrapper = super.load(path) + + class RandomForestClassifierWrapperWriter(instance: RandomForestClassifierWrapper) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("formula" -> instance.formula) ~ + ("features" -> instance.features.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + instance.pipeline.save(pipelinePath) + } + } + + class RandomForestClassifierWrapperReader extends MLReader[RandomForestClassifierWrapper] { + + override def load(path: String): RandomForestClassifierWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + val pipeline = PipelineModel.load(pipelinePath) + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val formula = (rMetadata \ "formula").extract[String] + val features = (rMetadata \ "features").extract[Array[String]] + + new RandomForestClassifierWrapper(pipeline, formula, features) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressionWrapper.scala new file mode 100644 index 000000000000..c8874407fa75 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressionWrapper.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.ml.r + +import org.apache.hadoop.fs.Path +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.ml.{Pipeline, PipelineModel} +import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.feature.RFormula +import org.apache.spark.ml.linalg.Vector +import org.apache.spark.ml.regression.{RandomForestRegressionModel, RandomForestRegressor} +import org.apache.spark.ml.util._ +import org.apache.spark.sql.{DataFrame, Dataset} + +private[r] class RandomForestRegressorWrapper private ( + val pipeline: PipelineModel, + val formula: String, + val features: Array[String]) extends MLWritable { + + private val DTModel: RandomForestRegressionModel = + pipeline.stages(1).asInstanceOf[RandomForestRegressionModel] + + lazy val numFeatures: Int = DTModel.numFeatures + lazy val featureImportances: Vector = DTModel.featureImportances + lazy val numTrees: Int = DTModel.getNumTrees + lazy val treeWeights: Array[Double] = DTModel.treeWeights + + def summary: String = DTModel.toDebugString + + def transform(dataset: Dataset[_]): DataFrame = { + pipeline.transform(dataset).drop(DTModel.getFeaturesCol) + } + + override def write: MLWriter = new + RandomForestRegressorWrapper.RandomForestRegressorWrapperWriter(this) +} + +private[r] object RandomForestRegressorWrapper extends MLReadable[RandomForestRegressorWrapper] { + def fit( // scalastyle:ignore + data: DataFrame, + formula: String, + maxDepth: Int, + maxBins: Int, + numTrees: Int, + impurity: String, + minInstancesPerNode: Int, + minInfoGain: Double, + checkpointInterval: Int, + featureSubsetStrategy: String, + seed: String, + subsamplingRate: Double, + maxMemoryInMB: Int, + cacheNodeIds: Boolean): RandomForestRegressorWrapper = { + + val rFormula = new RFormula() + .setFormula(formula) + RWrapperUtils.checkDataColumns(rFormula, data) + val rFormulaModel = rFormula.fit(data) + + // get feature names from output schema + val schema = rFormulaModel.transform(data).schema + val featureAttrs = AttributeGroup.fromStructField(schema(rFormulaModel.getFeaturesCol)) + .attributes.get + val features = featureAttrs.map(_.name.get) + + // assemble and fit the pipeline + val rfr = new RandomForestRegressor() + .setMaxDepth(maxDepth) + .setMaxBins(maxBins) + .setNumTrees(numTrees) + .setImpurity(impurity) + .setMinInstancesPerNode(minInstancesPerNode) + .setMinInfoGain(minInfoGain) + .setCheckpointInterval(checkpointInterval) + .setFeatureSubsetStrategy(featureSubsetStrategy) + .setSubsamplingRate(subsamplingRate) + .setMaxMemoryInMB(maxMemoryInMB) + .setCacheNodeIds(cacheNodeIds) + .setFeaturesCol(rFormula.getFeaturesCol) + if (seed != null && seed.length > 0) rfr.setSeed(seed.toLong) + + val pipeline = new Pipeline() + .setStages(Array(rFormulaModel, rfr)) + .fit(data) + + new RandomForestRegressorWrapper(pipeline, formula, features) + } + + override def read: MLReader[RandomForestRegressorWrapper] = new RandomForestRegressorWrapperReader + + override def load(path: String): RandomForestRegressorWrapper = super.load(path) + + class RandomForestRegressorWrapperWriter(instance: RandomForestRegressorWrapper) + extends MLWriter { + + override protected def saveImpl(path: String): Unit = { + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + + val rMetadata = ("class" -> instance.getClass.getName) ~ + ("formula" -> instance.formula) ~ + ("features" -> instance.features.toSeq) + val rMetadataJson: String = compact(render(rMetadata)) + + sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + instance.pipeline.save(pipelinePath) + } + } + + class RandomForestRegressorWrapperReader extends MLReader[RandomForestRegressorWrapper] { + + override def load(path: String): RandomForestRegressorWrapper = { + implicit val format = DefaultFormats + val rMetadataPath = new Path(path, "rMetadata").toString + val pipelinePath = new Path(path, "pipeline").toString + val pipeline = PipelineModel.load(pipelinePath) + + val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadata = parse(rMetadataStr) + val formula = (rMetadata \ "formula").extract[String] + val features = (rMetadata \ "features").extract[Array[String]] + + new RandomForestRegressorWrapper(pipeline, formula, features) + } + } +} From 7c3786929205b962b430cf7fc292602c2993c193 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Sun, 30 Oct 2016 16:21:37 -0700 Subject: [PATCH 007/132] [SPARK-18110][PYTHON][ML] add missing parameter in Python for RandomForest regression and classification ## What changes were proposed in this pull request? Add subsmaplingRate to randomForestClassifier Add varianceCol to randomForestRegressor In Python ## How was this patch tested? manual tests Author: Felix Cheung Closes #15638 from felixcheung/pyrandomforest. --- python/pyspark/ml/classification.py | 11 ++++++----- python/pyspark/ml/regression.py | 12 ++++++------ 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 3f763a10d406..d9ff356b9403 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -758,20 +758,21 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred probabilityCol="probability", rawPredictionCol="rawPrediction", maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", - numTrees=20, featureSubsetStrategy="auto", seed=None): + numTrees=20, featureSubsetStrategy="auto", seed=None, subsamplingRate=1.0): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ probabilityCol="probability", rawPredictionCol="rawPrediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="gini", \ - numTrees=20, featureSubsetStrategy="auto", seed=None) + numTrees=20, featureSubsetStrategy="auto", seed=None, subsamplingRate=1.0) """ super(RandomForestClassifier, self).__init__() self._java_obj = self._new_java_obj( "org.apache.spark.ml.classification.RandomForestClassifier", self.uid) self._setDefault(maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, - impurity="gini", numTrees=20, featureSubsetStrategy="auto") + impurity="gini", numTrees=20, featureSubsetStrategy="auto", + subsamplingRate=1.0) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -781,13 +782,13 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre probabilityCol="probability", rawPredictionCol="rawPrediction", maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=None, - impurity="gini", numTrees=20, featureSubsetStrategy="auto"): + impurity="gini", numTrees=20, featureSubsetStrategy="auto", subsamplingRate=1.0): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ probabilityCol="probability", rawPredictionCol="rawPrediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, seed=None, \ - impurity="gini", numTrees=20, featureSubsetStrategy="auto") + impurity="gini", numTrees=20, featureSubsetStrategy="auto", subsamplingRate=1.0) Sets params for linear classification. """ kwargs = self.setParams._input_kwargs diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 55d38033ef72..9233d2e7e1a7 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -594,7 +594,7 @@ class RandomForestParams(TreeEnsembleParams): featureSubsetStrategy = \ Param(Params._dummy(), "featureSubsetStrategy", "The number of features to consider for splits at each tree node. Supported " + - "options: " + ", ".join(supportedFeatureSubsetStrategies) + " (0.0-1.0], [1-n].", + "options: " + ", ".join(supportedFeatureSubsetStrategies) + ", (0.0-1.0], [1-n].", typeConverter=TypeConverters.toString) def __init__(self): @@ -828,7 +828,7 @@ def featureImportances(self): @inherit_doc class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol, HasSeed, RandomForestParams, TreeRegressorParams, HasCheckpointInterval, - JavaMLWritable, JavaMLReadable): + JavaMLWritable, JavaMLReadable, HasVarianceCol): """ `Random Forest `_ learning algorithm for regression. @@ -876,13 +876,13 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance", subsamplingRate=1.0, seed=None, numTrees=20, - featureSubsetStrategy="auto"): + featureSubsetStrategy="auto", varianceCol=None): """ __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, \ impurity="variance", subsamplingRate=1.0, seed=None, numTrees=20, \ - featureSubsetStrategy="auto") + featureSubsetStrategy="auto", varianceCol=None) """ super(RandomForestRegressor, self).__init__() self._java_obj = self._new_java_obj( @@ -900,13 +900,13 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, impurity="variance", subsamplingRate=1.0, seed=None, numTrees=20, - featureSubsetStrategy="auto"): + featureSubsetStrategy="auto", varianceCol=None): """ setParams(self, featuresCol="features", labelCol="label", predictionCol="prediction", \ maxDepth=5, maxBins=32, minInstancesPerNode=1, minInfoGain=0.0, \ maxMemoryInMB=256, cacheNodeIds=False, checkpointInterval=10, \ impurity="variance", subsamplingRate=1.0, seed=None, numTrees=20, \ - featureSubsetStrategy="auto") + featureSubsetStrategy="auto", varianceCol=None) Sets params for linear regression. """ kwargs = self.setParams._input_kwargs From d2923f173265b66a4ec71c3c86ff71a58d5aeb3d Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 31 Oct 2016 00:11:33 -0700 Subject: [PATCH 008/132] [SPARK-18143][SQL] Ignore Structured Streaming event logs to avoid breaking history server ## What changes were proposed in this pull request? Because of the refactoring work in Structured Streaming, the event logs generated by Strucutred Streaming in Spark 2.0.0 and 2.0.1 cannot be parsed. This PR just ignores these logs in ReplayListenerBus because no places use them. ## How was this patch tested? - Generated events logs using Spark 2.0.0 and 2.0.1, and saved them as `structured-streaming-query-event-logs-2.0.0.txt` and `structured-streaming-query-event-logs-2.0.1.txt` - The new added test makes sure ReplayListenerBus will skip these bad jsons. Author: Shixiong Zhu Closes #15663 from zsxwing/fix-event-log. --- .../spark/scheduler/ReplayListenerBus.scala | 13 ++++++ .../query-event-logs-version-2.0.0.txt | 4 ++ .../query-event-logs-version-2.0.1.txt | 4 ++ .../StreamingQueryListenerSuite.scala | 42 +++++++++++++++++++ 4 files changed, 63 insertions(+) create mode 100644 sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.0.txt create mode 100644 sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.1.txt diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 3eff8d952bfd..2424586431aa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -72,6 +72,10 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { postToAll(JsonProtocol.sparkEventFromJson(parse(currentLine))) } catch { + case e: ClassNotFoundException if KNOWN_REMOVED_CLASSES.contains(e.getMessage) => + // Ignore events generated by Structured Streaming in Spark 2.0.0 and 2.0.1. + // It's safe since no place uses them. + logWarning(s"Dropped incompatible Structured Streaming log: $currentLine") case jpe: JsonParseException => // We can only ignore exception from last line of the file that might be truncated // the last entry may not be the very last line in the event log, but we treat it @@ -102,4 +106,13 @@ private[spark] object ReplayListenerBus { // utility filter that selects all event logs during replay val SELECT_ALL_FILTER: ReplayEventsFilter = { (eventString: String) => true } + + /** + * Classes that were removed. Structured Streaming doesn't use them any more. However, parsing + * old json may fail and we can just ignore these failures. + */ + val KNOWN_REMOVED_CLASSES = Set( + "org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgress", + "org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated" + ) } diff --git a/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.0.txt b/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.0.txt new file mode 100644 index 000000000000..aa7e9a8c20c4 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.0.txt @@ -0,0 +1,4 @@ +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgress","queryInfo":{"name":"hello","id":0,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0"}],"sinkStatus":{"description":"org.apache.spark.sql.execution.streaming.MemorySink@2b85b3a5","offsetDesc":"[#0]"}}} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated","queryInfo":{"name":"hello","id":0,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0"}],"sinkStatus":{"description":"org.apache.spark.sql.execution.streaming.MemorySink@2b85b3a5","offsetDesc":"[#0]"}},"exception":null,"stackTrace":[]} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated","queryInfo":{"name":"hello","id":0,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0"}],"sinkStatus":{"description":"org.apache.spark.sql.execution.streaming.MemorySink@514502dc","offsetDesc":"[-]"}},"exception":"Query hello terminated with exception: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): java.lang.ArithmeticException: / by zero\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)\n\tat org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:246)\n\tat org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:240)\n\tat org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:784)\n\tat org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:784)\n\tat org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)\n\tat org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)\n\tat org.apache.spark.rdd.RDD.iterator(RDD.scala:283)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:85)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:","stackTrace":[{"methodName":"org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches","fileName":"StreamExecution.scala","lineNumber":208,"className":"org.apache.spark.sql.execution.streaming.StreamExecution","nativeMethod":false},{"methodName":"run","fileName":"StreamExecution.scala","lineNumber":120,"className":"org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1","nativeMethod":false}]} +{"Event":"SparkListenerApplicationEnd","Timestamp":1477593059313} diff --git a/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.1.txt b/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.1.txt new file mode 100644 index 000000000000..646cf107183b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/query-event-logs-version-2.0.1.txt @@ -0,0 +1,4 @@ +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryProgress","queryInfo":{"name":"hello","id":0,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0"}],"sinkStatus":{"description":"org.apache.spark.sql.execution.streaming.MemorySink@10e5ec94","offsetDesc":"[#0]"}}} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated","queryInfo":{"name":"hello","id":0,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0"}],"sinkStatus":{"description":"org.apache.spark.sql.execution.streaming.MemorySink@10e5ec94","offsetDesc":"[#0]"}},"exception":null} +{"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminated","queryInfo":{"name":"hello","id":0,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0"}],"sinkStatus":{"description":"org.apache.spark.sql.execution.streaming.MemorySink@70c61dc8","offsetDesc":"[-]"}},"exception":"org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): java.lang.ArithmeticException: / by zero\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)\n\tat org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:246)\n\tat org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:240)\n\tat org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)\n\tat org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)\n\tat org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)\n\tat org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)\n\tat org.apache.spark.rdd.RDD.iterator(RDD.scala:283)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:86)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:\n\tat org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1454)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1442)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1441)\n\tat scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)\n\tat scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)\n\tat org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1441)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)\n\tat org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:811)\n\tat scala.Option.foreach(Option.scala:257)\n\tat org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:811)\n\tat org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1667)\n\tat org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1622)\n\tat org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1611)\n\tat org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)\n\tat org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:632)\n\tat org.apache.spark.SparkContext.runJob(SparkContext.scala:1890)\n\tat org.apache.spark.SparkContext.runJob(SparkContext.scala:1903)\n\tat org.apache.spark.SparkContext.runJob(SparkContext.scala:1916)\n\tat org.apache.spark.SparkContext.runJob(SparkContext.scala:1930)\n\tat org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:912)\n\tat org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)\n\tat org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)\n\tat org.apache.spark.rdd.RDD.withScope(RDD.scala:358)\n\tat org.apache.spark.rdd.RDD.collect(RDD.scala:911)\n\tat org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:290)\n\tat org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2193)\n\tat org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:57)\n\tat org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2546)\n\tat org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$execute$1(Dataset.scala:2192)\n\tat org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2197)\n\tat org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$collect$1.apply(Dataset.scala:2197)\n\tat org.apache.spark.sql.Dataset.withCallback(Dataset.scala:2559)\n\tat org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collect(Dataset.scala:2197)\n\tat org.apache.spark.sql.Dataset.collect(Dataset.scala:2173)\n\tat org.apache.spark.sql.execution.streaming.MemorySink.addBatch(memory.scala:154)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatch(StreamExecution.scala:366)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1.apply$mcZ$sp(StreamExecution.scala:197)\n\tat org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:43)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:187)\n\tat org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:124)\nCaused by: java.lang.ArithmeticException: / by zero\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat $line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:25)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370)\n\tat org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:246)\n\tat org.apache.spark.sql.execution.SparkPlan$$anonfun$4.apply(SparkPlan.scala:240)\n\tat org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)\n\tat org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)\n\tat org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)\n\tat org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)\n\tat org.apache.spark.rdd.RDD.iterator(RDD.scala:283)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:86)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n"} +{"Event":"SparkListenerApplicationEnd","Timestamp":1477701734609} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index ff843865a017..cebb32a0a56c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark.sql.streaming +import scala.collection.mutable + import org.scalactic.TolerantNumerics import org.scalatest.BeforeAndAfter import org.scalatest.PrivateMethodTester._ import org.apache.spark.SparkException +import org.apache.spark.scheduler._ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.functions._ @@ -206,6 +209,45 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(queryQueryTerminated.exception === newQueryTerminated.exception) } + test("ReplayListenerBus should ignore broken event jsons generated in 2.0.0") { + // query-event-logs-version-2.0.0.txt has all types of events generated by + // Structured Streaming in Spark 2.0.0. + // SparkListenerApplicationEnd is the only valid event and it's the last event. We use it + // to verify that we can skip broken jsons generated by Structured Streaming. + testReplayListenerBusWithBorkenEventJsons("query-event-logs-version-2.0.0.txt") + } + + test("ReplayListenerBus should ignore broken event jsons generated in 2.0.1") { + // query-event-logs-version-2.0.1.txt has all types of events generated by + // Structured Streaming in Spark 2.0.1. + // SparkListenerApplicationEnd is the only valid event and it's the last event. We use it + // to verify that we can skip broken jsons generated by Structured Streaming. + testReplayListenerBusWithBorkenEventJsons("query-event-logs-version-2.0.1.txt") + } + + private def testReplayListenerBusWithBorkenEventJsons(fileName: String): Unit = { + val input = getClass.getResourceAsStream(s"/structured-streaming/$fileName") + val events = mutable.ArrayBuffer[SparkListenerEvent]() + try { + val replayer = new ReplayListenerBus() { + // Redirect all parsed events to `events` + override def doPostEvent( + listener: SparkListenerInterface, + event: SparkListenerEvent): Unit = { + events += event + } + } + // Add a dummy listener so that "doPostEvent" will be called. + replayer.addListener(new SparkListener {}) + replayer.replay(input, fileName) + // SparkListenerApplicationEnd is the only valid event + assert(events.size === 1) + assert(events(0).isInstanceOf[SparkListenerApplicationEnd]) + } finally { + input.close() + } + } + private def assertStreamingQueryInfoEquals( expected: StreamingQueryStatus, actual: StreamingQueryStatus): Unit = { From 26b07f1908eeffd934b1e86fb4de02f69945e004 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 31 Oct 2016 10:10:22 +0000 Subject: [PATCH 009/132] [BUILD] Close stale Pull Requests. Closes #11610 Closes #15411 Closes #15501 Closes #12613 Closes #12518 Closes #12026 Closes #15524 Closes #12693 Closes #12358 Closes #15588 Closes #15635 Closes #15678 Closes #14699 Closes #9008 Author: Sean Owen Closes #15685 from srowen/CloseStalePRs. From 8bfc3b7aac577e36aadc4fe6dee0665d0b2ae919 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 31 Oct 2016 13:39:59 -0700 Subject: [PATCH 010/132] [SPARK-17972][SQL] Add Dataset.checkpoint() to truncate large query plans ## What changes were proposed in this pull request? ### Problem Iterative ML code may easily create query plans that grow exponentially. We found that query planning time also increases exponentially even when all the sub-plan trees are cached. The following snippet illustrates the problem: ``` scala (0 until 6).foldLeft(Seq(1, 2, 3).toDS) { (plan, iteration) => println(s"== Iteration $iteration ==") val time0 = System.currentTimeMillis() val joined = plan.join(plan, "value").join(plan, "value").join(plan, "value").join(plan, "value") joined.cache() println(s"Query planning takes ${System.currentTimeMillis() - time0} ms") joined.as[Int] } // == Iteration 0 == // Query planning takes 9 ms // == Iteration 1 == // Query planning takes 26 ms // == Iteration 2 == // Query planning takes 53 ms // == Iteration 3 == // Query planning takes 163 ms // == Iteration 4 == // Query planning takes 700 ms // == Iteration 5 == // Query planning takes 3418 ms ``` This is because when building a new Dataset, the new plan is always built upon `QueryExecution.analyzed`, which doesn't leverage existing cached plans. On the other hand, usually, doing caching every a few iterations may not be the right direction for this problem since caching is too memory consuming (imaging computing connected components over a graph with 50 billion nodes). What we really need here is to truncate both the query plan (to minimize query planning time) and the lineage of the underlying RDD (to avoid stack overflow). ### Changes introduced in this PR This PR tries to fix this issue by introducing a `checkpoint()` method into `Dataset[T]`, which does exactly the things described above. The following snippet, which is essentially the same as the one above but invokes `checkpoint()` instead of `cache()`, shows the micro benchmark result of this PR: One key point is that the checkpointed Dataset should preserve the origianl partitioning and ordering information of the original Dataset, so that we can avoid unnecessary shuffling (similar to reading from a pre-bucketed table). This is done by adding `outputPartitioning` and `outputOrdering` to `LogicalRDD` and `RDDScanExec`. ### Micro benchmark ``` scala spark.sparkContext.setCheckpointDir("/tmp/cp") (0 until 100).foldLeft(Seq(1, 2, 3).toDS) { (plan, iteration) => println(s"== Iteration $iteration ==") val time0 = System.currentTimeMillis() val cp = plan.checkpoint() cp.count() System.out.println(s"Checkpointing takes ${System.currentTimeMillis() - time0} ms") val time1 = System.currentTimeMillis() val joined = cp.join(cp, "value").join(cp, "value").join(cp, "value").join(cp, "value") val result = joined.as[Int] println(s"Query planning takes ${System.currentTimeMillis() - time1} ms") result } // == Iteration 0 == // Checkpointing takes 591 ms // Query planning takes 13 ms // == Iteration 1 == // Checkpointing takes 1605 ms // Query planning takes 16 ms // == Iteration 2 == // Checkpointing takes 782 ms // Query planning takes 8 ms // == Iteration 3 == // Checkpointing takes 729 ms // Query planning takes 10 ms // == Iteration 4 == // Checkpointing takes 734 ms // Query planning takes 9 ms // == Iteration 5 == // ... // == Iteration 50 == // Checkpointing takes 571 ms // Query planning takes 7 ms // == Iteration 51 == // Checkpointing takes 548 ms // Query planning takes 7 ms // == Iteration 52 == // Checkpointing takes 596 ms // Query planning takes 8 ms // == Iteration 53 == // Checkpointing takes 568 ms // Query planning takes 7 ms // ... ``` You may see that although checkpointing is more heavy weight an operation, it always takes roughly the same amount of time to perform both checkpointing and query planning. ### Open question mengxr mentioned that it would be more convenient if we can make `Dataset.checkpoint()` eager, i.e., always performs a `RDD.count()` after calling `RDD.checkpoint()`. Not quite sure whether this is a universal requirement. Maybe we can add a `eager: Boolean` argument for `Dataset.checkpoint()` to support that. ## How was this patch tested? Unit test added in `DatasetSuite`. Author: Cheng Lian Author: Yin Huai Closes #15651 from liancheng/ds-checkpoint. --- .../scala/org/apache/spark/sql/Dataset.scala | 57 +++++++++++++++- .../spark/sql/execution/ExistingRDD.scala | 37 ++++++++-- .../spark/sql/execution/SparkStrategies.scala | 7 +- .../org/apache/spark/sql/DatasetSuite.scala | 68 +++++++++++++++++++ 4 files changed, 157 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 286d8549bfe2..6e0a2471e0fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -40,13 +40,14 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.optimizer.CombineUnions import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection} import org.apache.spark.sql.catalyst.util.usePrettyExpression import org.apache.spark.sql.execution.{FileRelation, LogicalRDD, QueryExecution, SQLExecution} import org.apache.spark.sql.execution.command.{CreateViewCommand, ExplainCommand, GlobalTempView, LocalTempView} -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.json.JacksonGenerator import org.apache.spark.sql.execution.python.EvaluatePython -import org.apache.spark.sql.streaming.{DataStreamWriter, StreamingQuery} +import org.apache.spark.sql.streaming.DataStreamWriter import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -482,6 +483,58 @@ class Dataset[T] private[sql]( @InterfaceStability.Evolving def isStreaming: Boolean = logicalPlan.isStreaming + /** + * Returns a checkpointed version of this Dataset. + * + * @group basic + * @since 2.1.0 + */ + @Experimental + @InterfaceStability.Evolving + def checkpoint(): Dataset[T] = checkpoint(eager = true) + + /** + * Returns a checkpointed version of this Dataset. + * + * @param eager When true, materializes the underlying checkpointed RDD eagerly. + * + * @group basic + * @since 2.1.0 + */ + @Experimental + @InterfaceStability.Evolving + def checkpoint(eager: Boolean): Dataset[T] = { + val internalRdd = queryExecution.toRdd.map(_.copy()) + internalRdd.checkpoint() + + if (eager) { + internalRdd.count() + } + + val physicalPlan = queryExecution.executedPlan + + // Takes the first leaf partitioning whenever we see a `PartitioningCollection`. Otherwise the + // size of `PartitioningCollection` may grow exponentially for queries involving deep inner + // joins. + def firstLeafPartitioning(partitioning: Partitioning): Partitioning = { + partitioning match { + case p: PartitioningCollection => firstLeafPartitioning(p.partitionings.head) + case p => p + } + } + + val outputPartitioning = firstLeafPartitioning(physicalPlan.outputPartitioning) + + Dataset.ofRows( + sparkSession, + LogicalRDD( + logicalPlan.output, + internalRdd, + outputPartitioning, + physicalPlan.outputOrdering + )(sparkSession)).as[T] + } + /** * Displays the Dataset in a tabular form. Strings more than 20 characters will be truncated, * and all cells will be aligned right. For example: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index d3a22228623e..455fb5bfbb6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.DataType import org.apache.spark.util.Utils @@ -130,17 +130,40 @@ case class ExternalRDDScanExec[T]( /** Logical plan node for scanning data from an RDD of InternalRow. */ case class LogicalRDD( output: Seq[Attribute], - rdd: RDD[InternalRow])(session: SparkSession) + rdd: RDD[InternalRow], + outputPartitioning: Partitioning = UnknownPartitioning(0), + outputOrdering: Seq[SortOrder] = Nil)(session: SparkSession) extends LeafNode with MultiInstanceRelation { override protected final def otherCopyArgs: Seq[AnyRef] = session :: Nil - override def newInstance(): LogicalRDD.this.type = - LogicalRDD(output.map(_.newInstance()), rdd)(session).asInstanceOf[this.type] + override def newInstance(): LogicalRDD.this.type = { + val rewrite = output.zip(output.map(_.newInstance())).toMap + + val rewrittenPartitioning = outputPartitioning match { + case p: Expression => + p.transform { + case e: Attribute => rewrite.getOrElse(e, e) + }.asInstanceOf[Partitioning] + + case p => p + } + + val rewrittenOrdering = outputOrdering.map(_.transform { + case e: Attribute => rewrite.getOrElse(e, e) + }.asInstanceOf[SortOrder]) + + LogicalRDD( + output.map(rewrite), + rdd, + rewrittenPartitioning, + rewrittenOrdering + )(session).asInstanceOf[this.type] + } override def sameResult(plan: LogicalPlan): Boolean = { plan.canonicalized match { - case LogicalRDD(_, otherRDD) => rdd.id == otherRDD.id + case LogicalRDD(_, otherRDD, _, _) => rdd.id == otherRDD.id case _ => false } } @@ -158,7 +181,9 @@ case class LogicalRDD( case class RDDScanExec( output: Seq[Attribute], rdd: RDD[InternalRow], - override val nodeName: String) extends LeafExecNode { + override val nodeName: String, + override val outputPartitioning: Partitioning = UnknownPartitioning(0), + override val outputOrdering: Seq[SortOrder] = Nil) extends LeafExecNode { override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 7cfae5ce283b..5412aca95dcf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -32,8 +32,6 @@ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight} import org.apache.spark.sql.execution.streaming.{MemoryPlan, StreamingExecutionRelation, StreamingRelation, StreamingRelationExec} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.streaming.StreamingQuery /** * Converts a logical plan into zero or more SparkPlans. This API is exposed for experimenting @@ -402,13 +400,14 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { generator, join = join, outer = outer, g.output, planLater(child)) :: Nil case logical.OneRowRelation => execution.RDDScanExec(Nil, singleRowRdd, "OneRowRelation") :: Nil - case r : logical.Range => + case r: logical.Range => execution.RangeExec(r) :: Nil case logical.RepartitionByExpression(expressions, child, nPartitions) => exchange.ShuffleExchange(HashPartitioning( expressions, nPartitions.getOrElse(numPartitions)), planLater(child)) :: Nil case ExternalRDD(outputObjAttr, rdd) => ExternalRDDScanExec(outputObjAttr, rdd) :: Nil - case LogicalRDD(output, rdd) => RDDScanExec(output, rdd, "ExistingRDD") :: Nil + case r: LogicalRDD => + RDDScanExec(r.output, r.rdd, "ExistingRDD", r.outputPartitioning, r.outputOrdering) :: Nil case BroadcastHint(child) => planLater(child) :: Nil case _ => Nil } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index cc367acae2ba..55f04878052a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -22,8 +22,11 @@ import java.sql.{Date, Timestamp} import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder} import org.apache.spark.sql.catalyst.util.sideBySide +import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec, SortExec} +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchange} import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} @@ -919,6 +922,71 @@ class DatasetSuite extends QueryTest with SharedSQLContext { df.withColumn("b", expr("0")).as[ClassData] .groupByKey(_.a).flatMapGroups { case (x, iter) => List[Int]() }) } + + Seq(true, false).foreach { eager => + def testCheckpointing(testName: String)(f: => Unit): Unit = { + test(s"Dataset.checkpoint() - $testName (eager = $eager)") { + withTempDir { dir => + val originalCheckpointDir = spark.sparkContext.checkpointDir + + try { + spark.sparkContext.setCheckpointDir(dir.getCanonicalPath) + f + } finally { + // Since the original checkpointDir can be None, we need + // to set the variable directly. + spark.sparkContext.checkpointDir = originalCheckpointDir + } + } + } + } + + testCheckpointing("basic") { + val ds = spark.range(10).repartition('id % 2).filter('id > 5).orderBy('id.desc) + val cp = ds.checkpoint(eager) + + val logicalRDD = cp.logicalPlan match { + case plan: LogicalRDD => plan + case _ => + val treeString = cp.logicalPlan.treeString(verbose = true) + fail(s"Expecting a LogicalRDD, but got\n$treeString") + } + + val dsPhysicalPlan = ds.queryExecution.executedPlan + val cpPhysicalPlan = cp.queryExecution.executedPlan + + assertResult(dsPhysicalPlan.outputPartitioning) { logicalRDD.outputPartitioning } + assertResult(dsPhysicalPlan.outputOrdering) { logicalRDD.outputOrdering } + + assertResult(dsPhysicalPlan.outputPartitioning) { cpPhysicalPlan.outputPartitioning } + assertResult(dsPhysicalPlan.outputOrdering) { cpPhysicalPlan.outputOrdering } + + // For a lazy checkpoint() call, the first check also materializes the checkpoint. + checkDataset(cp, (9L to 6L by -1L).map(java.lang.Long.valueOf): _*) + + // Reads back from checkpointed data and check again. + checkDataset(cp, (9L to 6L by -1L).map(java.lang.Long.valueOf): _*) + } + + testCheckpointing("should preserve partitioning information") { + val ds = spark.range(10).repartition('id % 2) + val cp = ds.checkpoint(eager) + + val agg = cp.groupBy('id % 2).agg(count('id)) + + agg.queryExecution.executedPlan.collectFirst { + case ShuffleExchange(_, _: RDDScanExec, _) => + case BroadcastExchangeExec(_, _: RDDScanExec) => + }.foreach { _ => + fail( + "No Exchange should be inserted above RDDScanExec since the checkpointed Dataset " + + "preserves partitioning information:\n\n" + agg.queryExecution + ) + } + + checkAnswer(agg, ds.groupBy('id % 2).agg(count('id))) + } + } } case class Generic[T](id: T, value: Double) From de3f87fa712c305fdd463fc36acffc5418c95c4d Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 31 Oct 2016 16:05:17 -0700 Subject: [PATCH 011/132] [SPARK-18030][TESTS] Fix flaky FileStreamSourceSuite by not deleting the files ## What changes were proposed in this pull request? The test `when schema inference is turned on, should read partition data` should not delete files because the source maybe is listing files. This PR just removes the delete actions since they are not necessary. ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #15699 from zsxwing/SPARK-18030. --- .../spark/sql/streaming/FileStreamSourceSuite.scala | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 47018b3a3c49..fab7642994ff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -102,12 +102,6 @@ class FileStreamSourceTest extends StreamTest with SharedSQLContext with Private } } - case class DeleteFile(file: File) extends ExternalAction { - def runAction(): Unit = { - Utils.deleteRecursively(file) - } - } - /** Use `format` and `path` to create FileStreamSource via DataFrameReader */ def createFileStream( format: String, @@ -697,10 +691,6 @@ class FileStreamSourceSuite extends FileStreamSourceTest { AddTextFileData("{'value': 'keep5'}", partitionBarSubDir, tmp), CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar"), ("keep5", "bar")), - // Delete the two partition dirs - DeleteFile(partitionFooSubDir), - DeleteFile(partitionBarSubDir), - AddTextFileData("{'value': 'keep6'}", partitionBarSubDir, tmp), CheckAnswer(("keep2", "foo"), ("keep3", "foo"), ("keep4", "bar"), ("keep5", "bar"), ("keep6", "bar")) From 6633b97b579c7f003d60b6bfa2e2a248340d3dc6 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 31 Oct 2016 16:26:52 -0700 Subject: [PATCH 012/132] [SPARK-18167][SQL] Also log all partitions when the SQLQuerySuite test flakes ## What changes were proposed in this pull request? One possibility for this test flaking is that we have corrupted the partition schema somehow in the tests, which causes the cast to decimal to fail in the call. This should at least show us the actual partition values. ## How was this patch tested? Run it locally, it prints out something like `ArrayBuffer(test(partcol=0), test(partcol=1), test(partcol=2), test(partcol=3), test(partcol=4))`. Author: Eric Liang Closes #15701 from ericl/print-more-info. --- .../main/scala/org/apache/spark/sql/hive/client/HiveShim.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 4bbbd66132b7..85edaf63db88 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -594,9 +594,8 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { // SPARK-18167 retry to investigate the flaky test. This should be reverted before // the release is cut. val retry = Try(getPartitionsByFilterMethod.invoke(hive, table, filter)) - val full = Try(getAllPartitionsMethod.invoke(hive, table)) logError("getPartitionsByFilter failed, retry success = " + retry.isSuccess) - logError("getPartitionsByFilter failed, full fetch success = " + full.isSuccess) + logError("all partitions: " + getAllPartitions(hive, table)) throw e } } From efc254a82bc3331d78023f00d29d4c4318dfb734 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 31 Oct 2016 19:46:55 -0700 Subject: [PATCH 013/132] [SPARK-18087][SQL] Optimize insert to not require REPAIR TABLE ## What changes were proposed in this pull request? When inserting into datasource tables with partitions managed by the hive metastore, we need to notify the metastore of newly added partitions. Previously this was implemented via `msck repair table`, but this is more expensive than needed. This optimizes the insertion path to add only the updated partitions. ## How was this patch tested? Existing tests (I verified manually that tests fail if the repair operation is omitted). Author: Eric Liang Closes #15633 from ericl/spark-18087. --- .../execution/datasources/DataSource.scala | 2 +- .../datasources/DataSourceStrategy.scala | 27 ++++++++++------- .../InsertIntoHadoopFsRelationCommand.scala | 3 +- .../datasources/PartitioningUtils.scala | 12 ++++++++ .../execution/datasources/WriteOutput.scala | 29 +++++++++++++------ 5 files changed, 52 insertions(+), 21 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index 996109865fdc..d980e6a15aab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -528,7 +528,7 @@ case class DataSource( columns, bucketSpec, format, - () => Unit, // No existing table needs to be refreshed. + _ => Unit, // No existing table needs to be refreshed. options, data.logicalPlan, mode) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index f0bcf94eadc9..34b77cab65de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.{CatalystConf, CatalystTypeConverters, Inte import org.apache.spark.sql.catalyst.CatalystTypeConverters.convertToScala import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SimpleCatalogRelation} +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation @@ -34,7 +35,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Union} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} -import org.apache.spark.sql.execution.command.{AlterTableRecoverPartitionsCommand, DDLUtils, ExecutedCommandExec} +import org.apache.spark.sql.execution.command.{AlterTableAddPartitionCommand, DDLUtils, ExecutedCommandExec} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -179,24 +180,30 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { "Cannot overwrite a path that is also being read from.") } + def refreshPartitionsCallback(updatedPartitions: Seq[TablePartitionSpec]): Unit = { + if (l.catalogTable.isDefined && + l.catalogTable.get.partitionColumnNames.nonEmpty && + l.catalogTable.get.partitionProviderIsHive) { + val metastoreUpdater = AlterTableAddPartitionCommand( + l.catalogTable.get.identifier, + updatedPartitions.map(p => (p, None)), + ifNotExists = true) + metastoreUpdater.run(t.sparkSession) + } + t.location.refresh() + } + val insertCmd = InsertIntoHadoopFsRelationCommand( outputPath, query.resolve(t.partitionSchema, t.sparkSession.sessionState.analyzer.resolver), t.bucketSpec, t.fileFormat, - () => t.location.refresh(), + refreshPartitionsCallback, t.options, query, mode) - if (l.catalogTable.isDefined && l.catalogTable.get.partitionColumnNames.nonEmpty && - l.catalogTable.get.partitionProviderIsHive) { - // TODO(ekl) we should be more efficient here and only recover the newly added partitions - val recoverPartitionCmd = AlterTableRecoverPartitionsCommand(l.catalogTable.get.identifier) - Union(insertCmd, recoverPartitionCmd) - } else { - insertCmd - } + insertCmd } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 22dbe7149531..a1221d0ae6d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.command.RunnableCommand @@ -40,7 +41,7 @@ case class InsertIntoHadoopFsRelationCommand( partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], fileFormat: FileFormat, - refreshFunction: () => Unit, + refreshFunction: (Seq[TablePartitionSpec]) => Unit, options: Map[String, String], @transient query: LogicalPlan, mode: SaveMode) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index f66e8b4e2b55..b51b41869bf0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -30,6 +30,7 @@ import org.apache.hadoop.util.Shell import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.Resolver +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Cast, Literal} import org.apache.spark.sql.types._ @@ -244,6 +245,17 @@ object PartitioningUtils { } } + /** + * Given a partition path fragment, e.g. `fieldOne=1/fieldTwo=2`, returns a parsed spec + * for that fragment, e.g. `Map(("fieldOne", "1"), ("fieldTwo", "2"))`. + */ + def parsePathFragment(pathFragment: String): TablePartitionSpec = { + pathFragment.split("/").map { kv => + val pair = kv.split("=", 2) + (unescapePathName(pair(0)), unescapePathName(pair(1))) + }.toMap + } + /** * Normalize the column names in partition specification, w.r.t. the real partition column names * and case sensitivity. e.g., if the partition spec has a column named `monTh`, and there is a diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala index bd56e511d0cc..0eb86fdd6caa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.datasources import java.util.{Date, UUID} +import scala.collection.mutable + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ @@ -30,6 +32,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.InternalRow @@ -85,7 +88,7 @@ object WriteOutput extends Logging { hadoopConf: Configuration, partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], - refreshFunction: () => Unit, + refreshFunction: (Seq[TablePartitionSpec]) => Unit, options: Map[String, String], isAppend: Boolean): Unit = { @@ -120,7 +123,7 @@ object WriteOutput extends Logging { val committer = setupDriverCommitter(job, outputPath.toString, isAppend) try { - sparkSession.sparkContext.runJob(queryExecution.toRdd, + val updatedPartitions = sparkSession.sparkContext.runJob(queryExecution.toRdd, (taskContext: TaskContext, iter: Iterator[InternalRow]) => { executeTask( description = description, @@ -128,11 +131,11 @@ object WriteOutput extends Logging { sparkPartitionId = taskContext.partitionId(), sparkAttemptNumber = taskContext.attemptNumber(), iterator = iter) - }) + }).flatten.distinct committer.commitJob(job) logInfo(s"Job ${job.getJobID} committed.") - refreshFunction() + refreshFunction(updatedPartitions.map(PartitioningUtils.parsePathFragment)) } catch { case cause: Throwable => logError(s"Aborting job ${job.getJobID}.", cause) committer.abortJob(job, JobStatus.State.FAILED) @@ -147,7 +150,7 @@ object WriteOutput extends Logging { sparkStageId: Int, sparkPartitionId: Int, sparkAttemptNumber: Int, - iterator: Iterator[InternalRow]): Unit = { + iterator: Iterator[InternalRow]): Set[String] = { val jobId = SparkHadoopWriter.createJobID(new Date, sparkStageId) val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId) @@ -187,11 +190,12 @@ object WriteOutput extends Logging { try { Utils.tryWithSafeFinallyAndFailureCallbacks(block = { // Execute the task to write rows out - writeTask.execute(iterator) + val outputPaths = writeTask.execute(iterator) writeTask.releaseResources() // Commit the task SparkHadoopMapRedUtil.commitTask(committer, taskAttemptContext, jobId.getId, taskId.getId) + outputPaths })(catchBlock = { // If there is an error, release resource and then abort the task try { @@ -213,7 +217,7 @@ object WriteOutput extends Logging { * automatically trigger task aborts. */ private trait ExecuteWriteTask { - def execute(iterator: Iterator[InternalRow]): Unit + def execute(iterator: Iterator[InternalRow]): Set[String] def releaseResources(): Unit final def filePrefix(split: Int, uuid: String, bucketId: Option[Int]): String = { @@ -240,11 +244,12 @@ object WriteOutput extends Logging { outputWriter } - override def execute(iter: Iterator[InternalRow]): Unit = { + override def execute(iter: Iterator[InternalRow]): Set[String] = { while (iter.hasNext) { val internalRow = iter.next() outputWriter.writeInternal(internalRow) } + Set.empty } override def releaseResources(): Unit = { @@ -327,7 +332,7 @@ object WriteOutput extends Logging { newWriter } - override def execute(iter: Iterator[InternalRow]): Unit = { + override def execute(iter: Iterator[InternalRow]): Set[String] = { // We should first sort by partition columns, then bucket id, and finally sorting columns. val sortingExpressions: Seq[Expression] = description.partitionColumns ++ bucketIdExpression ++ sortColumns @@ -375,6 +380,7 @@ object WriteOutput extends Logging { // If anything below fails, we should abort the task. var currentKey: UnsafeRow = null + val updatedPartitions = mutable.Set[String]() while (sortedIterator.next()) { val nextKey = getBucketingKey(sortedIterator.getKey).asInstanceOf[UnsafeRow] if (currentKey != nextKey) { @@ -386,6 +392,10 @@ object WriteOutput extends Logging { logDebug(s"Writing partition: $currentKey") currentWriter = newOutputWriter(currentKey, getPartitionString) + val partitionPath = getPartitionString(currentKey).getString(0) + if (partitionPath.nonEmpty) { + updatedPartitions.add(partitionPath) + } } currentWriter.writeInternal(sortedIterator.getValue) } @@ -393,6 +403,7 @@ object WriteOutput extends Logging { currentWriter.close() currentWriter = null } + updatedPartitions.toSet } override def releaseResources(): Unit = { From 7d6c87155c740cf622c2c600a8ca64154d24c422 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 31 Oct 2016 20:23:22 -0700 Subject: [PATCH 014/132] [SPARK-18167][SQL] Retry when the SQLQuerySuite test flakes ## What changes were proposed in this pull request? This will re-run the flaky test a few times after it fails. This will help determine if it's due to nondeterministic test setup, or because of some environment issue (e.g. leaked config from another test). cc yhuai Author: Eric Liang Closes #15708 from ericl/spark-18167-3. --- .../sql/hive/execution/SQLQuerySuite.scala | 28 +++++++++++++------ 1 file changed, 20 insertions(+), 8 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 2735d3a5267e..f64010a64b01 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -1566,14 +1566,26 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("SPARK-10562: partition by column with mixed case name") { - withTable("tbl10562") { - val df = Seq(2012 -> "a").toDF("Year", "val") - df.write.partitionBy("Year").saveAsTable("tbl10562") - checkAnswer(sql("SELECT year FROM tbl10562"), Row(2012)) - checkAnswer(sql("SELECT Year FROM tbl10562"), Row(2012)) - checkAnswer(sql("SELECT yEAr FROM tbl10562"), Row(2012)) - checkAnswer(sql("SELECT val FROM tbl10562 WHERE Year > 2015"), Nil) - checkAnswer(sql("SELECT val FROM tbl10562 WHERE Year == 2012"), Row("a")) + def runOnce() { + withTable("tbl10562") { + val df = Seq(2012 -> "a").toDF("Year", "val") + df.write.partitionBy("Year").saveAsTable("tbl10562") + checkAnswer(sql("SELECT year FROM tbl10562"), Row(2012)) + checkAnswer(sql("SELECT Year FROM tbl10562"), Row(2012)) + checkAnswer(sql("SELECT yEAr FROM tbl10562"), Row(2012)) + checkAnswer(sql("SELECT val FROM tbl10562 WHERE Year > 2015"), Nil) + checkAnswer(sql("SELECT val FROM tbl10562 WHERE Year == 2012"), Row("a")) + } + } + try { + runOnce() + } catch { + case t: Throwable => + // Retry to gather more test data. TODO(ekl) revert this once we deflake this test. + runOnce() + runOnce() + runOnce() + throw t } } From d9d1465009fb40550467089ede315496552374c5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 31 Oct 2016 22:23:38 -0700 Subject: [PATCH 015/132] [SPARK-18024][SQL] Introduce an internal commit protocol API ## What changes were proposed in this pull request? This patch introduces an internal commit protocol API that is used by the batch data source to do write commits. It currently has only one implementation that uses Hadoop MapReduce's OutputCommitter API. In the future, this commit API can be used to unify streaming and batch commits. ## How was this patch tested? Should be covered by existing write tests. Author: Reynold Xin Author: Eric Liang Closes #15707 from rxin/SPARK-18024-2. --- .../ml/source/libsvm/LibSVMRelation.scala | 17 +- .../datasources/FileCommitProtocol.scala | 254 ++++++++++++++++++ .../execution/datasources/OutputWriter.scala | 26 +- .../execution/datasources/WriteOutput.scala | 167 +++--------- .../datasources/csv/CSVRelation.scala | 17 +- .../datasources/json/JsonFileFormat.scala | 17 +- .../parquet/ParquetFileFormat.scala | 8 +- .../parquet/ParquetOutputWriter.scala | 19 +- .../datasources/text/TextFileFormat.scala | 17 +- .../apache/spark/sql/internal/SQLConf.scala | 29 +- .../spark/sql/hive/orc/OrcFileFormat.scala | 28 +- .../sql/sources/CommitFailureTestSource.scala | 10 +- .../sql/sources/SimpleTextRelation.scala | 19 +- 13 files changed, 387 insertions(+), 241 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCommitProtocol.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala index 5e9e6ff1a569..cb3ca1b6c4be 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMRelation.scala @@ -41,17 +41,11 @@ import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration private[libsvm] class LibSVMOutputWriter( - stagingDir: String, - fileNamePrefix: String, + path: String, dataSchema: StructType, context: TaskAttemptContext) extends OutputWriter { - override val path: String = { - val compressionExtension = TextOutputWriter.getCompressionExtension(context) - new Path(stagingDir, fileNamePrefix + ".libsvm" + compressionExtension).toString - } - private[this] val buffer = new Text() private val recordWriter: RecordWriter[NullWritable, Text] = { @@ -135,11 +129,14 @@ private[libsvm] class LibSVMFileFormat extends TextBasedFileFormat with DataSour dataSchema: StructType): OutputWriterFactory = { new OutputWriterFactory { override def newInstance( - stagingDir: String, - fileNamePrefix: String, + path: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new LibSVMOutputWriter(stagingDir, fileNamePrefix, dataSchema, context) + new LibSVMOutputWriter(path, dataSchema, context) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + ".libsvm" + TextOutputWriter.getCompressionExtension(context) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCommitProtocol.scala new file mode 100644 index 000000000000..1ce9ae4266c1 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCommitProtocol.scala @@ -0,0 +1,254 @@ +/* + * 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 + +import java.util.{Date, UUID} + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl + +import org.apache.spark.SparkHadoopWriter +import org.apache.spark.internal.Logging +import org.apache.spark.mapred.SparkHadoopMapRedUtil +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.Utils + + +object FileCommitProtocol { + class TaskCommitMessage(obj: Any) extends Serializable + + object EmptyTaskCommitMessage extends TaskCommitMessage(Unit) + + /** + * Instantiates a FileCommitProtocol using the given className. + */ + def instantiate(className: String, outputPath: String, isAppend: Boolean): FileCommitProtocol = { + try { + val clazz = Utils.classForName(className).asInstanceOf[Class[FileCommitProtocol]] + + // First try the one with argument (outputPath: String, isAppend: Boolean). + // If that doesn't exist, try the one with (outputPath: String). + try { + val ctor = clazz.getDeclaredConstructor(classOf[String], classOf[Boolean]) + ctor.newInstance(outputPath, isAppend.asInstanceOf[java.lang.Boolean]) + } catch { + case _: NoSuchMethodException => + val ctor = clazz.getDeclaredConstructor(classOf[String]) + ctor.newInstance(outputPath) + } + } catch { + case e: ClassNotFoundException => + throw e + } + } +} + + +/** + * An interface to define how a Spark job commits its outputs. Implementations must be serializable, + * as the committer instance instantiated on the driver will be used for tasks on executors. + * + * The proper call sequence is: + * + * 1. Driver calls setupJob. + * 2. As part of each task's execution, executor calls setupTask and then commitTask + * (or abortTask if task failed). + * 3. When all necessary tasks completed successfully, the driver calls commitJob. If the job + * failed to execute (e.g. too many failed tasks), the job should call abortJob. + */ +abstract class FileCommitProtocol { + import FileCommitProtocol._ + + /** + * Setups up a job. Must be called on the driver before any other methods can be invoked. + */ + def setupJob(jobContext: JobContext): Unit + + /** + * Commits a job after the writes succeed. Must be called on the driver. + */ + def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit + + /** + * Aborts a job after the writes fail. Must be called on the driver. + * + * Calling this function is a best-effort attempt, because it is possible that the driver + * just crashes (or killed) before it can call abort. + */ + def abortJob(jobContext: JobContext): Unit + + /** + * Sets up a task within a job. + * Must be called before any other task related methods can be invoked. + */ + def setupTask(taskContext: TaskAttemptContext): Unit + + /** + * Notifies the commit protocol to add a new file, and gets back the full path that should be + * used. Must be called on the executors when running tasks. + * + * Note that the returned temp file may have an arbitrary path. The commit protocol only + * promises that the file will be at the location specified by the arguments after job commit. + * + * A full file path consists of the following parts: + * 1. the base path + * 2. some sub-directory within the base path, used to specify partitioning + * 3. file prefix, usually some unique job id with the task id + * 4. bucket id + * 5. source specific file extension, e.g. ".snappy.parquet" + * + * The "dir" parameter specifies 2, and "ext" parameter specifies both 4 and 5, and the rest + * are left to the commit protocol implementation to decide. + */ + def newTaskTempFile(taskContext: TaskAttemptContext, dir: Option[String], ext: String): String + + /** + * Commits a task after the writes succeed. Must be called on the executors when running tasks. + */ + def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage + + /** + * Aborts a task after the writes have failed. Must be called on the executors when running tasks. + * + * Calling this function is a best-effort attempt, because it is possible that the executor + * just crashes (or killed) before it can call abort. + */ + def abortTask(taskContext: TaskAttemptContext): Unit +} + + +/** + * An [[FileCommitProtocol]] implementation backed by an underlying Hadoop OutputCommitter + * (from the newer mapreduce API, not the old mapred API). + * + * Unlike Hadoop's OutputCommitter, this implementation is serializable. + */ +class HadoopCommitProtocolWrapper(path: String, isAppend: Boolean) + extends FileCommitProtocol with Serializable with Logging { + + import FileCommitProtocol._ + + /** OutputCommitter from Hadoop is not serializable so marking it transient. */ + @transient private var committer: OutputCommitter = _ + + /** UUID used to identify the job in file name. */ + private val uuid: String = UUID.randomUUID().toString + + private def setupCommitter(context: TaskAttemptContext): Unit = { + committer = context.getOutputFormatClass.newInstance().getOutputCommitter(context) + + if (!isAppend) { + // If we are appending data to an existing dir, we will only use the output committer + // associated with the file output format since it is not safe to use a custom + // committer for appending. For example, in S3, direct parquet output committer may + // leave partial data in the destination dir when the appending job fails. + // See SPARK-8578 for more details. + val configuration = context.getConfiguration + val clazz = + configuration.getClass(SQLConf.OUTPUT_COMMITTER_CLASS.key, null, classOf[OutputCommitter]) + + if (clazz != null) { + logInfo(s"Using user defined output committer class ${clazz.getCanonicalName}") + + // Every output format based on org.apache.hadoop.mapreduce.lib.output.OutputFormat + // has an associated output committer. To override this output committer, + // we will first try to use the output committer set in SQLConf.OUTPUT_COMMITTER_CLASS. + // If a data source needs to override the output committer, it needs to set the + // output committer in prepareForWrite method. + if (classOf[FileOutputCommitter].isAssignableFrom(clazz)) { + // The specified output committer is a FileOutputCommitter. + // So, we will use the FileOutputCommitter-specified constructor. + val ctor = clazz.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext]) + committer = ctor.newInstance(new Path(path), context) + } else { + // The specified output committer is just an OutputCommitter. + // So, we will use the no-argument constructor. + val ctor = clazz.getDeclaredConstructor() + committer = ctor.newInstance() + } + } + } + logInfo(s"Using output committer class ${committer.getClass.getCanonicalName}") + } + + override def newTaskTempFile( + taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = { + // The file name looks like part-r-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003.gz.parquet + // Note that %05d does not truncate the split number, so if we have more than 100000 tasks, + // the file name is fine and won't overflow. + val split = taskContext.getTaskAttemptID.getTaskID.getId + val filename = f"part-$split%05d-$uuid$ext" + + val stagingDir: String = committer match { + // For FileOutputCommitter it has its own staging path called "work path". + case f: FileOutputCommitter => Option(f.getWorkPath.toString).getOrElse(path) + case _ => path + } + + dir.map { d => + new Path(new Path(stagingDir, d), filename).toString + }.getOrElse { + new Path(stagingDir, filename).toString + } + } + + override def setupJob(jobContext: JobContext): Unit = { + // Setup IDs + val jobId = SparkHadoopWriter.createJobID(new Date, 0) + val taskId = new TaskID(jobId, TaskType.MAP, 0) + val taskAttemptId = new TaskAttemptID(taskId, 0) + + // Set up the configuration object + jobContext.getConfiguration.set("mapred.job.id", jobId.toString) + jobContext.getConfiguration.set("mapred.tip.id", taskAttemptId.getTaskID.toString) + jobContext.getConfiguration.set("mapred.task.id", taskAttemptId.toString) + jobContext.getConfiguration.setBoolean("mapred.task.is.map", true) + jobContext.getConfiguration.setInt("mapred.task.partition", 0) + + val taskAttemptContext = new TaskAttemptContextImpl(jobContext.getConfiguration, taskAttemptId) + setupCommitter(taskAttemptContext) + + committer.setupJob(jobContext) + } + + override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = { + committer.commitJob(jobContext) + } + + override def abortJob(jobContext: JobContext): Unit = { + committer.abortJob(jobContext, JobStatus.State.FAILED) + } + + override def setupTask(taskContext: TaskAttemptContext): Unit = { + setupCommitter(taskContext) + committer.setupTask(taskContext) + } + + override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = { + val attemptId = taskContext.getTaskAttemptID + SparkHadoopMapRedUtil.commitTask( + committer, taskContext, attemptId.getJobID.getId, attemptId.getTaskID.getId) + EmptyTaskCommitMessage + } + + override def abortTask(taskContext: TaskAttemptContext): Unit = { + committer.abortTask(taskContext) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala index fbf6e96d3f85..a73c8146c1b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala @@ -30,28 +30,21 @@ import org.apache.spark.sql.types.StructType * to executor side to create actual [[OutputWriter]]s on the fly. */ abstract class OutputWriterFactory extends Serializable { + + /** Returns the file extension to be used when writing files out. */ + def getFileExtension(context: TaskAttemptContext): String + /** * When writing to a [[HadoopFsRelation]], this method gets called by each task on executor side * to instantiate new [[OutputWriter]]s. * - * @param stagingDir Base path (directory) of the file to which this [[OutputWriter]] is supposed - * to write. Note that this may not point to the final output file. For - * example, `FileOutputFormat` writes to temporary directories and then merge - * written files back to the final destination. In this case, `path` points to - * a temporary output file under the temporary directory. - * @param fileNamePrefix Prefix of the file name. The returned OutputWriter must make sure this - * prefix is used in the actual file name. For example, if the prefix is - * "part-1-2-3", then the file name must start with "part_1_2_3" but can - * end in arbitrary extension that is deterministic given the configuration - * (i.e. the suffix extension should not depend on any task id, attempt id, - * or partition id). + * @param path Path to write the file. * @param dataSchema Schema of the rows to be written. Partition columns are not included in the * schema if the relation being written is partitioned. * @param context The Hadoop MapReduce task context. */ def newInstance( - stagingDir: String, - fileNamePrefix: String, + path: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter @@ -77,13 +70,6 @@ abstract class OutputWriterFactory extends Serializable { * executor side. This instance is used to persist rows to this single output file. */ abstract class OutputWriter { - - /** - * The path of the file to be written out. This path should include the staging directory and - * the file name prefix passed into the associated createOutputWriter function. - */ - def path: String - /** * Persists a single row. Invoked on the executor side. When writing to dynamically partitioned * tables, dynamic partition columns are not included in rows to be written. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala index 0eb86fdd6caa..a07855111b40 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala @@ -24,12 +24,11 @@ import scala.collection.mutable import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter, FileOutputFormat} +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.spark._ import org.apache.spark.internal.Logging -import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec @@ -38,7 +37,7 @@ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{SQLExecution, UnsafeKVExternalSorter} -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.execution.datasources.FileCommitProtocol.TaskCommitMessage import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.util.{SerializableConfiguration, Utils} import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter @@ -57,8 +56,7 @@ object WriteOutput extends Logging { val nonPartitionColumns: Seq[Attribute], val bucketSpec: Option[BucketSpec], val isAppend: Boolean, - val path: String, - val outputFormatClass: Class[_ <: OutputFormat[_, _]]) + val path: String) extends Serializable { assert(AttributeSet(allColumns) == AttributeSet(partitionColumns ++ nonPartitionColumns), @@ -114,31 +112,38 @@ object WriteOutput extends Logging { nonPartitionColumns = dataColumns, bucketSpec = bucketSpec, isAppend = isAppend, - path = outputPath.toString, - outputFormatClass = job.getOutputFormatClass) + path = outputPath.toString) SQLExecution.withNewExecutionId(sparkSession, queryExecution) { // This call shouldn't be put into the `try` block below because it only initializes and // prepares the job, any exception thrown from here shouldn't cause abortJob() to be called. - val committer = setupDriverCommitter(job, outputPath.toString, isAppend) + val committer = FileCommitProtocol.instantiate( + sparkSession.sessionState.conf.fileCommitProtocolClass, + outputPath.toString, + isAppend) + committer.setupJob(job) try { - val updatedPartitions = sparkSession.sparkContext.runJob(queryExecution.toRdd, + val ret = sparkSession.sparkContext.runJob(queryExecution.toRdd, (taskContext: TaskContext, iter: Iterator[InternalRow]) => { executeTask( description = description, sparkStageId = taskContext.stageId(), sparkPartitionId = taskContext.partitionId(), sparkAttemptNumber = taskContext.attemptNumber(), + committer, iterator = iter) - }).flatten.distinct + }) - committer.commitJob(job) + val commitMsgs = ret.map(_._1) + val updatedPartitions = ret.flatMap(_._2).distinct.map(PartitioningUtils.parsePathFragment) + + committer.commitJob(job, commitMsgs) logInfo(s"Job ${job.getJobID} committed.") - refreshFunction(updatedPartitions.map(PartitioningUtils.parsePathFragment)) + refreshFunction(updatedPartitions) } catch { case cause: Throwable => logError(s"Aborting job ${job.getJobID}.", cause) - committer.abortJob(job, JobStatus.State.FAILED) + committer.abortJob(job) throw new SparkException("Job aborted.", cause) } } @@ -150,7 +155,8 @@ object WriteOutput extends Logging { sparkStageId: Int, sparkPartitionId: Int, sparkAttemptNumber: Int, - iterator: Iterator[InternalRow]): Set[String] = { + committer: FileCommitProtocol, + iterator: Iterator[InternalRow]): (TaskCommitMessage, Set[String]) = { val jobId = SparkHadoopWriter.createJobID(new Date, sparkStageId) val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId) @@ -169,33 +175,21 @@ object WriteOutput extends Logging { new TaskAttemptContextImpl(hadoopConf, taskAttemptId) } - val committer = newOutputCommitter( - description.outputFormatClass, taskAttemptContext, description.path, description.isAppend) committer.setupTask(taskAttemptContext) - // Figure out where we need to write data to for staging. - // For FileOutputCommitter it has its own staging path called "work path". - val stagingPath = committer match { - case f: FileOutputCommitter => f.getWorkPath.toString - case _ => description.path - } - val writeTask = if (description.partitionColumns.isEmpty && description.bucketSpec.isEmpty) { - new SingleDirectoryWriteTask(description, taskAttemptContext, stagingPath) + new SingleDirectoryWriteTask(description, taskAttemptContext, committer) } else { - new DynamicPartitionWriteTask(description, taskAttemptContext, stagingPath) + new DynamicPartitionWriteTask(description, taskAttemptContext, committer) } try { Utils.tryWithSafeFinallyAndFailureCallbacks(block = { - // Execute the task to write rows out - val outputPaths = writeTask.execute(iterator) + // Execute the task to write rows out and commit the task. + val outputPartitions = writeTask.execute(iterator) writeTask.releaseResources() - - // Commit the task - SparkHadoopMapRedUtil.commitTask(committer, taskAttemptContext, jobId.getId, taskId.getId) - outputPaths + (committer.commitTask(taskAttemptContext), outputPartitions) })(catchBlock = { // If there is an error, release resource and then abort the task try { @@ -217,27 +211,28 @@ object WriteOutput extends Logging { * automatically trigger task aborts. */ private trait ExecuteWriteTask { + /** + * 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 - - final def filePrefix(split: Int, uuid: String, bucketId: Option[Int]): String = { - val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") - f"part-r-$split%05d-$uuid$bucketString" - } } /** Writes data to a single directory (used for non-dynamic-partition writes). */ private class SingleDirectoryWriteTask( description: WriteJobDescription, taskAttemptContext: TaskAttemptContext, - stagingPath: String) extends ExecuteWriteTask { + committer: FileCommitProtocol) extends ExecuteWriteTask { private[this] var outputWriter: OutputWriter = { - val split = taskAttemptContext.getTaskAttemptID.getTaskID.getId + val tmpFilePath = committer.newTaskTempFile( + taskAttemptContext, + None, + description.outputWriterFactory.getFileExtension(taskAttemptContext)) val outputWriter = description.outputWriterFactory.newInstance( - stagingDir = stagingPath, - fileNamePrefix = filePrefix(split, description.uuid, None), + path = tmpFilePath, dataSchema = description.nonPartitionColumns.toStructType, context = taskAttemptContext) outputWriter.initConverter(dataSchema = description.nonPartitionColumns.toStructType) @@ -267,7 +262,7 @@ object WriteOutput extends Logging { private class DynamicPartitionWriteTask( description: WriteJobDescription, taskAttemptContext: TaskAttemptContext, - stagingPath: String) extends ExecuteWriteTask { + committer: FileCommitProtocol) extends ExecuteWriteTask { // currentWriter is initialized whenever we see a new key private var currentWriter: OutputWriter = _ @@ -307,25 +302,20 @@ object WriteOutput extends Logging { * file extension, e.g. part-r-00009-ea518ad4-455a-4431-b471-d24e03814677-00002.gz.parquet */ private def newOutputWriter(key: InternalRow, partString: UnsafeProjection): OutputWriter = { - val path = - if (description.partitionColumns.nonEmpty) { - val partitionPath = partString(key).getString(0) - new Path(stagingPath, partitionPath).toString - } else { - stagingPath - } + val partDir = + if (description.partitionColumns.isEmpty) None else Option(partString(key).getString(0)) // If the bucket spec is defined, the bucket column is right after the partition columns val bucketId = if (description.bucketSpec.isDefined) { - Some(key.getInt(description.partitionColumns.length)) + BucketingUtils.bucketIdToString(key.getInt(description.partitionColumns.length)) } else { - None + "" } + val ext = bucketId + description.outputWriterFactory.getFileExtension(taskAttemptContext) - val split = taskAttemptContext.getTaskAttemptID.getTaskID.getId + val path = committer.newTaskTempFile(taskAttemptContext, partDir, ext) val newWriter = description.outputWriterFactory.newInstance( - stagingDir = path, - fileNamePrefix = filePrefix(split, description.uuid, bucketId), + path = path, dataSchema = description.nonPartitionColumns.toStructType, context = taskAttemptContext) newWriter.initConverter(description.nonPartitionColumns.toStructType) @@ -413,75 +403,4 @@ object WriteOutput extends Logging { } } } - - private def setupDriverCommitter(job: Job, path: String, isAppend: Boolean): OutputCommitter = { - // Setup IDs - val jobId = SparkHadoopWriter.createJobID(new Date, 0) - val taskId = new TaskID(jobId, TaskType.MAP, 0) - val taskAttemptId = new TaskAttemptID(taskId, 0) - - // Set up the configuration object - job.getConfiguration.set("mapred.job.id", jobId.toString) - job.getConfiguration.set("mapred.tip.id", taskAttemptId.getTaskID.toString) - job.getConfiguration.set("mapred.task.id", taskAttemptId.toString) - job.getConfiguration.setBoolean("mapred.task.is.map", true) - job.getConfiguration.setInt("mapred.task.partition", 0) - - val taskAttemptContext = new TaskAttemptContextImpl(job.getConfiguration, taskAttemptId) - val outputCommitter = newOutputCommitter( - job.getOutputFormatClass, taskAttemptContext, path, isAppend) - outputCommitter.setupJob(job) - outputCommitter - } - - private def newOutputCommitter( - outputFormatClass: Class[_ <: OutputFormat[_, _]], - context: TaskAttemptContext, - path: String, - isAppend: Boolean): OutputCommitter = { - val defaultOutputCommitter = outputFormatClass.newInstance().getOutputCommitter(context) - - if (isAppend) { - // If we are appending data to an existing dir, we will only use the output committer - // associated with the file output format since it is not safe to use a custom - // committer for appending. For example, in S3, direct parquet output committer may - // leave partial data in the destination dir when the appending job fails. - // See SPARK-8578 for more details - logInfo( - s"Using default output committer ${defaultOutputCommitter.getClass.getCanonicalName} " + - "for appending.") - defaultOutputCommitter - } else { - val configuration = context.getConfiguration - val clazz = - configuration.getClass(SQLConf.OUTPUT_COMMITTER_CLASS.key, null, classOf[OutputCommitter]) - - if (clazz != null) { - logInfo(s"Using user defined output committer class ${clazz.getCanonicalName}") - - // Every output format based on org.apache.hadoop.mapreduce.lib.output.OutputFormat - // has an associated output committer. To override this output committer, - // we will first try to use the output committer set in SQLConf.OUTPUT_COMMITTER_CLASS. - // If a data source needs to override the output committer, it needs to set the - // output committer in prepareForWrite method. - if (classOf[FileOutputCommitter].isAssignableFrom(clazz)) { - // The specified output committer is a FileOutputCommitter. - // So, we will use the FileOutputCommitter-specified constructor. - val ctor = clazz.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext]) - ctor.newInstance(new Path(path), context) - } else { - // The specified output committer is just an OutputCommitter. - // So, we will use the no-argument constructor. - val ctor = clazz.getDeclaredConstructor() - ctor.newInstance() - } - } else { - // If output committer class is not set, we will use the one associated with the - // file output format. - logInfo( - s"Using output committer class ${defaultOutputCommitter.getClass.getCanonicalName}") - defaultOutputCommitter - } - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala index a35cfdb2c234..a249b9d9d59b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala @@ -171,26 +171,23 @@ object CSVRelation extends Logging { private[csv] class CSVOutputWriterFactory(params: CSVOptions) extends OutputWriterFactory { override def newInstance( - stagingDir: String, - fileNamePrefix: String, + path: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new CsvOutputWriter(stagingDir, fileNamePrefix, dataSchema, context, params) + new CsvOutputWriter(path, dataSchema, context, params) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + ".csv" + TextOutputWriter.getCompressionExtension(context) } } private[csv] class CsvOutputWriter( - stagingDir: String, - fileNamePrefix: String, + path: String, dataSchema: StructType, context: TaskAttemptContext, params: CSVOptions) extends OutputWriter with Logging { - override val path: String = { - val compressionExtension = TextOutputWriter.getCompressionExtension(context) - new Path(stagingDir, fileNamePrefix + ".csv" + compressionExtension).toString - } - // create the Generator without separator inserted between 2 records private[this] val text = new Text() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index 651fa78a4e92..5a409c04c929 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -83,11 +83,14 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { new OutputWriterFactory { override def newInstance( - stagingDir: String, - fileNamePrefix: String, + path: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new JsonOutputWriter(stagingDir, parsedOptions, fileNamePrefix, dataSchema, context) + new JsonOutputWriter(path, parsedOptions, dataSchema, context) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + ".json" + TextOutputWriter.getCompressionExtension(context) } } } @@ -154,18 +157,12 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { } private[json] class JsonOutputWriter( - stagingDir: String, + path: String, options: JSONOptions, - fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext) extends OutputWriter with Logging { - override val path: String = { - val compressionExtension = TextOutputWriter.getCompressionExtension(context) - new Path(stagingDir, fileNamePrefix + ".json" + compressionExtension).toString - } - private[this] val writer = new CharArrayWriter() // create the Generator without separator inserted between 2 records private[this] val gen = new JacksonGenerator(dataSchema, writer, options) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 502dd0e8d4cf..77c83ba38efe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -33,6 +33,7 @@ import org.apache.parquet.{Log => ApacheParquetLog} import org.apache.parquet.filter2.compat.FilterCompat import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.hadoop._ +import org.apache.parquet.hadoop.codec.CodecConfig import org.apache.parquet.hadoop.util.ContextUtil import org.apache.parquet.schema.MessageType import org.slf4j.bridge.SLF4JBridgeHandler @@ -133,10 +134,13 @@ class ParquetFileFormat new OutputWriterFactory { override def newInstance( path: String, - fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new ParquetOutputWriter(path, fileNamePrefix, context) + new ParquetOutputWriter(path, context) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + CodecConfig.from(context).getCodec.getExtension + ".parquet" } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala index 1300069c42b0..92d4f27be3fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala @@ -89,7 +89,7 @@ private[parquet] class ParquetOutputWriterFactory( * Returns a [[OutputWriter]] that writes data to the give path without using * [[OutputCommitter]]. */ - override def newWriter(path1: String): OutputWriter = new OutputWriter { + override def newWriter(path: String): OutputWriter = new OutputWriter { // Create TaskAttemptContext that is used to pass on Configuration to the ParquetRecordWriter private val hadoopTaskAttemptId = new TaskAttemptID(new TaskID(new JobID, TaskType.MAP, 0), 0) @@ -99,8 +99,6 @@ private[parquet] class ParquetOutputWriterFactory( // Instance of ParquetRecordWriter that does not use OutputCommitter private val recordWriter = createNoCommitterRecordWriter(path, hadoopAttemptContext) - override def path: String = path1 - override def write(row: Row): Unit = { throw new UnsupportedOperationException("call writeInternal") } @@ -127,27 +125,22 @@ private[parquet] class ParquetOutputWriterFactory( /** Disable the use of the older API. */ override def newInstance( path: String, - fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { throw new UnsupportedOperationException("this version of newInstance not supported for " + "ParquetOutputWriterFactory") } + + override def getFileExtension(context: TaskAttemptContext): String = { + CodecConfig.from(context).getCodec.getExtension + ".parquet" + } } // NOTE: This class is instantiated and used on executor side only, no need to be serializable. -private[parquet] class ParquetOutputWriter( - stagingDir: String, - fileNamePrefix: String, - context: TaskAttemptContext) +private[parquet] class ParquetOutputWriter(path: String, context: TaskAttemptContext) extends OutputWriter { - override val path: String = { - val filename = fileNamePrefix + CodecConfig.from(context).getCodec.getExtension + ".parquet" - new Path(stagingDir, filename).toString - } - private val recordWriter: RecordWriter[Void, InternalRow] = { new ParquetOutputFormat[InternalRow]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index d40b5725199a..8e043960326d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala @@ -75,11 +75,14 @@ class TextFileFormat extends TextBasedFileFormat with DataSourceRegister { new OutputWriterFactory { override def newInstance( - stagingDir: String, - fileNamePrefix: String, + path: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new TextOutputWriter(stagingDir, fileNamePrefix, dataSchema, context) + new TextOutputWriter(path, dataSchema, context) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + ".txt" + TextOutputWriter.getCompressionExtension(context) } } } @@ -124,17 +127,11 @@ class TextFileFormat extends TextBasedFileFormat with DataSourceRegister { } class TextOutputWriter( - stagingDir: String, - fileNamePrefix: String, + path: String, dataSchema: StructType, context: TaskAttemptContext) extends OutputWriter { - override val path: String = { - val compressionExtension = TextOutputWriter.getCompressionExtension(context) - new Path(stagingDir, fileNamePrefix + ".txt" + compressionExtension).toString - } - private[this] val buffer = new Text() private val recordWriter: RecordWriter[NullWritable, Text] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index dc31f3bc323f..29e79847aa38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -30,6 +30,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.network.util.ByteUnit import org.apache.spark.sql.catalyst.CatalystConf +import org.apache.spark.sql.execution.datasources.HadoopCommitProtocolWrapper import org.apache.spark.util.Utils //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -240,9 +241,8 @@ object SQLConf { val PARQUET_OUTPUT_COMMITTER_CLASS = SQLConfigBuilder("spark.sql.parquet.output.committer.class") .doc("The output committer class used by Parquet. The specified class needs to be a " + "subclass of org.apache.hadoop.mapreduce.OutputCommitter. Typically, it's also a subclass " + - "of org.apache.parquet.hadoop.ParquetOutputCommitter. NOTE: 1. Instead of SQLConf, this " + - "option must be set in Hadoop Configuration. 2. This option overrides " + - "\"spark.sql.sources.outputCommitterClass\".") + "of org.apache.parquet.hadoop.ParquetOutputCommitter.") + .internal() .stringConf .createWithDefault(classOf[ParquetOutputCommitter].getName) @@ -375,16 +375,17 @@ object SQLConf { .booleanConf .createWithDefault(true) - // The output committer class used by HadoopFsRelation. The specified class needs to be a + // The output committer class used by data sources. The specified class needs to be a // subclass of org.apache.hadoop.mapreduce.OutputCommitter. - // - // NOTE: - // - // 1. Instead of SQLConf, this option *must be set in Hadoop Configuration*. - // 2. This option can be overridden by "spark.sql.parquet.output.committer.class". val OUTPUT_COMMITTER_CLASS = SQLConfigBuilder("spark.sql.sources.outputCommitterClass").internal().stringConf.createOptional + val FILE_COMMIT_PROTOCOL_CLASS = + SQLConfigBuilder("spark.sql.sources.commitProtocolClass") + .internal() + .stringConf + .createWithDefault(classOf[HadoopCommitProtocolWrapper].getName) + val PARALLEL_PARTITION_DISCOVERY_THRESHOLD = SQLConfigBuilder("spark.sql.sources.parallelPartitionDiscovery.threshold") .doc("The maximum number of files allowed for listing files at driver side. If the number " + @@ -518,6 +519,12 @@ object SQLConf { .booleanConf .createWithDefault(true) + val STREAMING_FILE_COMMIT_PROTOCOL_CLASS = + SQLConfigBuilder("spark.sql.streaming.commitProtocolClass") + .internal() + .stringConf + .createWithDefault(classOf[HadoopCommitProtocolWrapper].getName) + val FILE_SINK_LOG_DELETION = SQLConfigBuilder("spark.sql.streaming.fileSink.log.deletion") .internal() .doc("Whether to delete the expired log files in file stream sink.") @@ -631,6 +638,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def isUnsupportedOperationCheckEnabled: Boolean = getConf(UNSUPPORTED_OPERATION_CHECK_ENABLED) + def streamingFileCommitProtocolClass: String = getConf(STREAMING_FILE_COMMIT_PROTOCOL_CLASS) + def fileSinkLogDeletion: Boolean = getConf(FILE_SINK_LOG_DELETION) def fileSinkLogCompactInterval: Int = getConf(FILE_SINK_LOG_COMPACT_INTERVAL) @@ -741,6 +750,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def partitionColumnTypeInferenceEnabled: Boolean = getConf(SQLConf.PARTITION_COLUMN_TYPE_INFERENCE) + def fileCommitProtocolClass: String = getConf(SQLConf.FILE_COMMIT_PROTOCOL_CLASS) + def parallelPartitionDiscoveryThreshold: Int = getConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index eba7aa386ade..7c519a074317 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -83,11 +83,19 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable new OutputWriterFactory { override def newInstance( - stagingDir: String, - fileNamePrefix: String, + path: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new OrcOutputWriter(stagingDir, fileNamePrefix, dataSchema, context) + new OrcOutputWriter(path, dataSchema, context) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + val compressionExtension: String = { + val name = context.getConfiguration.get(OrcRelation.ORC_COMPRESSION) + OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "") + } + + compressionExtension + ".orc" } } } @@ -210,23 +218,11 @@ private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration) } private[orc] class OrcOutputWriter( - stagingDir: String, - fileNamePrefix: String, + path: String, dataSchema: StructType, context: TaskAttemptContext) extends OutputWriter { - override val path: String = { - val compressionExtension: String = { - val name = context.getConfiguration.get(OrcRelation.ORC_COMPRESSION) - OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "") - } - // It has the `.orc` extension at the end because (de)compression tools - // such as gunzip would not be able to decompress this as the compression - // is not applied on this whole file but on each "stream" in ORC format. - new Path(stagingDir, fileNamePrefix + compressionExtension + ".orc").toString - } - private[this] val serializer = new OrcSerializer(dataSchema, context.getConfiguration) // `OrcRecordWriter.close()` creates an empty file if no rows are written at all. We use this diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala index 731540db17ee..abc7c8cc4db8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.sources -import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.TaskContext @@ -40,19 +39,16 @@ class CommitFailureTestSource extends SimpleTextSource { dataSchema: StructType): OutputWriterFactory = new OutputWriterFactory { override def newInstance( - stagingDir: String, - fileNamePrefix: String, + path: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new SimpleTextOutputWriter(stagingDir, fileNamePrefix, context) { + new SimpleTextOutputWriter(path, context) { var failed = false TaskContext.get().addTaskFailureListener { (t: TaskContext, e: Throwable) => failed = true SimpleTextRelation.callbackCalled = true } - override val path: String = new Path(stagingDir, fileNamePrefix).toString - override def write(row: Row): Unit = { if (SimpleTextRelation.failWriter) { sys.error("Intentional task writer failure for testing purpose.") @@ -67,6 +63,8 @@ class CommitFailureTestSource extends SimpleTextSource { } } } + + override def getFileExtension(context: TaskAttemptContext): String = "" } override def shortName(): String = "commit-failure-test" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index 9896b9bde99c..64d0ecbeefc9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -51,12 +51,13 @@ class SimpleTextSource extends TextBasedFileFormat with DataSourceRegister { SimpleTextRelation.lastHadoopConf = Option(job.getConfiguration) new OutputWriterFactory { override def newInstance( - stagingDir: String, - fileNamePrefix: String, + path: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new SimpleTextOutputWriter(stagingDir, fileNamePrefix, context) + new SimpleTextOutputWriter(path, context) } + + override def getFileExtension(context: TaskAttemptContext): String = "" } } @@ -120,14 +121,11 @@ class SimpleTextSource extends TextBasedFileFormat with DataSourceRegister { } } -class SimpleTextOutputWriter( - stagingDir: String, fileNamePrefix: String, context: TaskAttemptContext) +class SimpleTextOutputWriter(path: String, context: TaskAttemptContext) extends OutputWriter { - override val path: String = new Path(stagingDir, fileNamePrefix).toString - private val recordWriter: RecordWriter[NullWritable, Text] = - new AppendingTextOutputFormat(new Path(stagingDir), fileNamePrefix).getRecordWriter(context) + new AppendingTextOutputFormat(path).getRecordWriter(context) override def write(row: Row): Unit = { val serialized = row.toSeq.map { v => @@ -141,15 +139,14 @@ class SimpleTextOutputWriter( } } -class AppendingTextOutputFormat(stagingDir: Path, fileNamePrefix: String) - extends TextOutputFormat[NullWritable, Text] { +class AppendingTextOutputFormat(path: String) extends TextOutputFormat[NullWritable, Text] { val numberFormat = NumberFormat.getInstance() numberFormat.setMinimumIntegerDigits(5) numberFormat.setGroupingUsed(false) override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - new Path(stagingDir, fileNamePrefix) + new Path(path) } } From dd85eb5448c8f2672260b57e94c0da0eaac12616 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 1 Nov 2016 00:24:08 -0700 Subject: [PATCH 016/132] [SPARK-18107][SQL] Insert overwrite statement runs much slower in spark-sql than it does in hive-client ## What changes were proposed in this pull request? As reported on the jira, insert overwrite statement runs much slower in Spark, compared with hive-client. It seems there is a patch [HIVE-11940](https://github.com/apache/hive/commit/ba21806b77287e237e1aa68fa169d2a81e07346d) which largely improves insert overwrite performance on Hive. HIVE-11940 is patched after Hive 2.0.0. Because Spark SQL uses older Hive library, we can not benefit from such improvement. The reporter verified that there is also a big performance gap between Hive 1.2.1 (520.037 secs) and Hive 2.0.1 (35.975 secs) on insert overwrite execution. Instead of upgrading to Hive 2.0 in Spark SQL, which might not be a trivial task, this patch provides an approach to delete the partition before asking Hive to load data files into the partition. Note: The case reported on the jira is insert overwrite to partition. Since `Hive.loadTable` also uses the function to replace files, insert overwrite to table should has the same issue. We can take the same approach to delete the table first. I will upgrade this to include this. ## How was this patch tested? Jenkins tests. There are existing tests using insert overwrite statement. Those tests should be passed. I added a new test to specially test insert overwrite into partition. For performance issue, as I don't have Hive 2.0 environment, this needs the reporter to verify it. Please refer to the jira. Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request. Author: Liang-Chi Hsieh Closes #15667 from viirya/improve-hive-insertoverwrite. --- .../hive/execution/InsertIntoHiveTable.scala | 24 +++++++++++++- .../sql/hive/execution/SQLQuerySuite.scala | 33 +++++++++++++++++++ 2 files changed, 56 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index c3c4e2925b90..2843100fb3b3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.command.{AlterTableAddPartitionCommand, AlterTableDropPartitionCommand} import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} import org.apache.spark.SparkException @@ -257,7 +258,28 @@ case class InsertIntoHiveTable( table.catalogTable.identifier.table, partitionSpec) + var doHiveOverwrite = overwrite + if (oldPart.isEmpty || !ifNotExists) { + // SPARK-18107: Insert overwrite runs much slower than hive-client. + // Newer Hive largely improves insert overwrite performance. As Spark uses older Hive + // version and we may not want to catch up new Hive version every time. We delete the + // Hive partition first and then load data file into the Hive partition. + if (oldPart.nonEmpty && overwrite) { + oldPart.get.storage.locationUri.map { uri => + val partitionPath = new Path(uri) + val fs = partitionPath.getFileSystem(hadoopConf) + if (fs.exists(partitionPath)) { + if (!fs.delete(partitionPath, true)) { + throw new RuntimeException( + "Cannot remove partition directory '" + partitionPath.toString) + } + // Don't let Hive do overwrite operation since it is slower. + doHiveOverwrite = false + } + } + } + // inheritTableSpecs is set to true. It should be set to false for an IMPORT query // which is currently considered as a Hive native command. val inheritTableSpecs = true @@ -266,7 +288,7 @@ case class InsertIntoHiveTable( table.catalogTable.identifier.table, outputPath.toString, partitionSpec, - isOverwrite = overwrite, + isOverwrite = doHiveOverwrite, holdDDLTime = holdDDLTime, inheritTableSpecs = inheritTableSpecs) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index f64010a64b01..8b916932ff54 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -1973,6 +1973,39 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("Insert overwrite with partition") { + withTable("tableWithPartition") { + sql( + """ + |CREATE TABLE tableWithPartition (key int, value STRING) + |PARTITIONED BY (part STRING) + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE tableWithPartition PARTITION (part = '1') + |SELECT * FROM default.src + """.stripMargin) + checkAnswer( + sql("SELECT part, key, value FROM tableWithPartition"), + sql("SELECT '1' AS part, key, value FROM default.src") + ) + + sql( + """ + |INSERT OVERWRITE TABLE tableWithPartition PARTITION (part = '1') + |SELECT * FROM VALUES (1, "one"), (2, "two"), (3, null) AS data(key, value) + """.stripMargin) + checkAnswer( + sql("SELECT part, key, value FROM tableWithPartition"), + sql( + """ + |SELECT '1' AS part, key, value FROM VALUES + |(1, "one"), (2, "two"), (3, null) AS data(key, value) + """.stripMargin) + ) + } + } + def testCommandAvailable(command: String): Boolean = { val attempt = Try(Process(command).run(ProcessLogger(_ => ())).exitValue()) attempt.isSuccess && attempt.get == 0 From 623fc7fc67735cfafdb7f527bd3df210987943c6 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 1 Nov 2016 13:08:49 +0000 Subject: [PATCH 017/132] [MINOR][DOC] Remove spaces following slashs ## What changes were proposed in this pull request? This PR merges multiple lines enumerating items in order to remove the redundant spaces following slashes in [Structured Streaming Programming Guide in 2.0.2-rc1](http://people.apache.org/~pwendell/spark-releases/spark-2.0.2-rc1-docs/structured-streaming-programming-guide.html). - Before: `Scala/ Java/ Python` - After: `Scala/Java/Python` ## How was this patch tested? Manual by the followings because this is documentation update. ``` cd docs SKIP_API=1 jekyll build ``` Author: Dongjoon Hyun Closes #15686 from dongjoon-hyun/minor_doc_space. --- .../structured-streaming-programming-guide.md | 44 +++++++++---------- 1 file changed, 20 insertions(+), 24 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 173fd6e8c73b..d838ed35a14f 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -14,10 +14,8 @@ Structured Streaming is a scalable and fault-tolerant stream processing engine b # Quick Example Let’s say you want to maintain a running word count of text data received from a data server listening on a TCP socket. Let’s see how you can express this using Structured Streaming. You can see the full code in -[Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala)/ -[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java)/ -[Python]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/sql/streaming/structured_network_wordcount.py). And if you -[download Spark](http://spark.apache.org/downloads.html), you can directly run the example. In any case, let’s walk through the example step-by-step and understand how it works. First, we have to import the necessary classes and create a local SparkSession, the starting point of all functionalities related to Spark. +[Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCount.scala)/[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCount.java)/[Python]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/sql/streaming/structured_network_wordcount.py). +And if you [download Spark](http://spark.apache.org/downloads.html), you can directly run the example. In any case, let’s walk through the example step-by-step and understand how it works. First, we have to import the necessary classes and create a local SparkSession, the starting point of all functionalities related to Spark.
@@ -409,16 +407,15 @@ Delivering end-to-end exactly-once semantics was one of key goals behind the des to track the read position in the stream. The engine uses checkpointing and write ahead logs to record the offset range of the data being processed in each trigger. The streaming sinks are designed to be idempotent for handling reprocessing. Together, using replayable sources and idempotent sinks, Structured Streaming can ensure **end-to-end exactly-once semantics** under any failure. # API using Datasets and DataFrames -Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as well as streaming, unbounded data. Similar to static Datasets/DataFrames, you can use the common entry point `SparkSession` ([Scala](api/scala/index.html#org.apache.spark.sql.SparkSession)/ -[Java](api/java/org/apache/spark/sql/SparkSession.html)/ -[Python](api/python/pyspark.sql.html#pyspark.sql.SparkSession) docs) to create streaming DataFrames/Datasets from streaming sources, and apply the same operations on them as static DataFrames/Datasets. If you are not familiar with Datasets/DataFrames, you are strongly advised to familiarize yourself with them using the +Since Spark 2.0, DataFrames and Datasets can represent static, bounded data, as well as streaming, unbounded data. Similar to static Datasets/DataFrames, you can use the common entry point `SparkSession` +([Scala](api/scala/index.html#org.apache.spark.sql.SparkSession)/[Java](api/java/org/apache/spark/sql/SparkSession.html)/[Python](api/python/pyspark.sql.html#pyspark.sql.SparkSession) docs) +to create streaming DataFrames/Datasets from streaming sources, and apply the same operations on them as static DataFrames/Datasets. If you are not familiar with Datasets/DataFrames, you are strongly advised to familiarize yourself with them using the [DataFrame/Dataset Programming Guide](sql-programming-guide.html). ## Creating streaming DataFrames and streaming Datasets Streaming DataFrames can be created through the `DataStreamReader` interface -([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamReader)/ -[Java](api/java/org/apache/spark/sql/streaming/DataStreamReader.html)/ -[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source – data format, schema, options, etc. +([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamReader)/[Java](api/java/org/apache/spark/sql/streaming/DataStreamReader.html)/[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamReader) docs) +returned by `SparkSession.readStream()`. Similar to the read interface for creating static DataFrame, you can specify the details of the source – data format, schema, options, etc. #### Data Sources In Spark 2.0, there are a few built-in sources. @@ -628,9 +625,7 @@ The result tables would look something like the following. ![Window Operations](img/structured-streaming-window.png) Since this windowing is similar to grouping, in code, you can use `groupBy()` and `window()` operations to express windowed aggregations. You can see the full code for the below examples in -[Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala)/ -[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCountWindowed.java)/ -[Python]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py). +[Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredNetworkWordCountWindowed.scala)/[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredNetworkWordCountWindowed.java)/[Python]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/sql/streaming/structured_network_wordcount_windowed.py).
@@ -753,10 +748,9 @@ In addition, there are some Dataset methods that will not work on streaming Data If you try any of these operations, you will see an AnalysisException like "operation XYZ is not supported with streaming DataFrames/Datasets". ## Starting Streaming Queries -Once you have defined the final result DataFrame/Dataset, all that is left is for you start the streaming computation. To do that, you have to use the -`DataStreamWriter` ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamWriter)/ -[Java](api/java/org/apache/spark/sql/streaming/DataStreamWriter.html)/ -[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamWriter) docs) returned through `Dataset.writeStream()`. You will have to specify one or more of the following in this interface. +Once you have defined the final result DataFrame/Dataset, all that is left is for you start the streaming computation. To do that, you have to use the `DataStreamWriter` +([Scala](api/scala/index.html#org.apache.spark.sql.streaming.DataStreamWriter)/[Java](api/java/org/apache/spark/sql/streaming/DataStreamWriter.html)/[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.DataStreamWriter) docs) +returned through `Dataset.writeStream()`. You will have to specify one or more of the following in this interface. - *Details of the output sink:* Data format, location, etc. @@ -953,8 +947,9 @@ spark.sql("select * from aggregates").show() # interactively query in-memory t
#### Using Foreach -The `foreach` operation allows arbitrary operations to be computed on the output data. As of Spark 2.0, this is available only for Scala and Java. To use this, you will have to implement the interface `ForeachWriter` ([Scala](api/scala/index.html#org.apache.spark.sql.ForeachWriter)/ -[Java](api/java/org/apache/spark/sql/ForeachWriter.html) docs), which has methods that get called whenever there is a sequence of rows generated as output after a trigger. Note the following important points. +The `foreach` operation allows arbitrary operations to be computed on the output data. As of Spark 2.0, this is available only for Scala and Java. To use this, you will have to implement the interface `ForeachWriter` +([Scala](api/scala/index.html#org.apache.spark.sql.ForeachWriter)/[Java](api/java/org/apache/spark/sql/ForeachWriter.html) docs), +which has methods that get called whenever there is a sequence of rows generated as output after a trigger. Note the following important points. - The writer must be serializable, as it will be serialized and sent to the executors for execution. @@ -1046,9 +1041,9 @@ query.sinkStatus() # progress information about data written to the output sin
-You can start any number of queries in a single SparkSession. They will all be running concurrently sharing the cluster resources. You can use `sparkSession.streams()` to get the `StreamingQueryManager` ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryManager)/ -[Java](api/java/org/apache/spark/sql/streaming/StreamingQueryManager.html)/ -[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.StreamingQueryManager) docs) that can be used to manage the currently active queries. +You can start any number of queries in a single SparkSession. They will all be running concurrently sharing the cluster resources. You can use `sparkSession.streams()` to get the `StreamingQueryManager` +([Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryManager)/[Java](api/java/org/apache/spark/sql/streaming/StreamingQueryManager.html)/[Python](api/python/pyspark.sql.html#pyspark.sql.streaming.StreamingQueryManager) docs) +that can be used to manage the currently active queries.
@@ -1092,8 +1087,9 @@ spark.streams().awaitAnyTermination() # block until any one of them terminates
-Finally, for asynchronous monitoring of streaming queries, you can create and attach a `StreamingQueryListener` ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryListener)/ -[Java](api/java/org/apache/spark/sql/streaming/StreamingQueryListener.html) docs), which will give you regular callback-based updates when queries are started and terminated. +Finally, for asynchronous monitoring of streaming queries, you can create and attach a `StreamingQueryListener` +([Scala](api/scala/index.html#org.apache.spark.sql.streaming.StreamingQueryListener)/[Java](api/java/org/apache/spark/sql/streaming/StreamingQueryListener.html) docs), +which will give you regular callback-based updates when queries are started and terminated. ## Recovering from Failures with Checkpointing In case of a failure or intentional shutdown, you can recover the previous progress and state of a previous query, and continue where it left off. This is done using checkpointing and write ahead logs. You can configure a query with a checkpoint location, and the query will save all the progress information (i.e. range of offsets processed in each trigger) and the running aggregates (e.g. word counts in the [quick example](#quick-example)) to the checkpoint location. As of Spark 2.0, this checkpoint location has to be a path in an HDFS compatible file system, and can be set as an option in the DataStreamWriter when [starting a query](#starting-streaming-queries). From cb80edc26349e2e358d27fe2ae8e5d6959b77fab Mon Sep 17 00:00:00 2001 From: wangzhenhua Date: Tue, 1 Nov 2016 13:11:24 +0000 Subject: [PATCH 018/132] [SPARK-18111][SQL] Wrong ApproximatePercentile answer when multiple records have the minimum value ## What changes were proposed in this pull request? When multiple records have the minimum value, the answer of ApproximatePercentile is wrong. ## How was this patch tested? add a test case Author: wangzhenhua Closes #15641 from wzhfy/percentile. --- .../spark/sql/catalyst/util/QuantileSummaries.scala | 4 +++- .../spark/sql/ApproximatePercentileQuerySuite.scala | 11 +++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala index 27928c493d5f..04f4ff2a9224 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala @@ -264,7 +264,9 @@ object QuantileSummaries { res.prepend(head) // If necessary, add the minimum element: val currHead = currentSamples.head - if (currHead.value < head.value) { + // don't add the minimum element if `currentSamples` has only one element (both `currHead` and + // `head` point to the same element) + if (currHead.value <= head.value && currentSamples.length > 1) { res.prepend(currentSamples.head) } res.toArray diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala index 37d7c442bbeb..e98092df4951 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala @@ -64,6 +64,17 @@ class ApproximatePercentileQuerySuite extends QueryTest with SharedSQLContext { } } + test("percentile_approx, multiple records with the minimum value in a partition") { + withTempView(table) { + spark.sparkContext.makeRDD(Seq(1, 1, 2, 1, 1, 3, 1, 1, 4, 1, 1, 5), 4).toDF("col") + .createOrReplaceTempView(table) + checkAnswer( + spark.sql(s"SELECT percentile_approx(col, array(0.5)) FROM $table"), + Row(Seq(1.0D)) + ) + } + } + test("percentile_approx, with different accuracies") { withTempView(table) { From e34b4e12673fb76c92f661d7c03527410857a0f8 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 1 Nov 2016 13:14:17 +0000 Subject: [PATCH 019/132] [SPARK-15994][MESOS] Allow enabling Mesos fetch cache in coarse executor backend Mesos 0.23.0 introduces a Fetch Cache feature http://mesos.apache.org/documentation/latest/fetcher/ which allows caching of resources specified in command URIs. This patch: - Updates the Mesos shaded protobuf dependency to 0.23.0 - Allows setting `spark.mesos.fetcherCache.enable` to enable the fetch cache for all specified URIs. (URIs must be specified for the setting to have any affect) - Updates documentation for Mesos configuration with the new setting. This patch does NOT: - Allow for per-URI caching configuration. The cache setting is global to ALL URIs for the command. Author: Charles Allen Closes #13713 from drcrallen/SPARK15994. --- docs/running-on-mesos.md | 9 ++++-- .../cluster/mesos/MesosClusterScheduler.scala | 3 +- .../MesosCoarseGrainedSchedulerBackend.scala | 6 ++-- .../cluster/mesos/MesosSchedulerUtils.scala | 6 ++-- ...osCoarseGrainedSchedulerBackendSuite.scala | 28 +++++++++++++++++++ 5 files changed, 45 insertions(+), 7 deletions(-) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 77b06fcf3374..923d8dbebf3d 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -506,8 +506,13 @@ See the [configuration page](configuration.html) for information on Spark config since this configuration is just a upper limit and not a guaranteed amount. - - + + spark.mesos.fetcherCache.enable + false + + If set to `true`, all URIs (example: `spark.executor.uri`, `spark.mesos.uris`) will be cached by the [Mesos fetcher cache](http://mesos.apache.org/documentation/latest/fetcher/) + + # Troubleshooting and Debugging diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 0b454997772d..635712c00d30 100644 --- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -129,6 +129,7 @@ private[spark] class MesosClusterScheduler( private val queuedCapacity = conf.getInt("spark.mesos.maxDrivers", 200) private val retainedDrivers = conf.getInt("spark.mesos.retainedDrivers", 200) private val maxRetryWaitTime = conf.getInt("spark.mesos.cluster.retry.wait.max", 60) // 1 minute + private val useFetchCache = conf.getBoolean("spark.mesos.fetchCache.enable", false) private val schedulerState = engineFactory.createEngine("scheduler") private val stateLock = new Object() private val finishedDrivers = @@ -396,7 +397,7 @@ private[spark] class MesosClusterScheduler( val jarUrl = desc.jarUrl.stripPrefix("file:").stripPrefix("local:") ((jarUrl :: confUris) ++ getDriverExecutorURI(desc).toList).map(uri => - CommandInfo.URI.newBuilder().setValue(uri.trim()).build()) + CommandInfo.URI.newBuilder().setValue(uri.trim()).setCache(useFetchCache).build()) } private def getDriverCommandValue(desc: MesosDriverDescription): String = { diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index e67bf3e328f9..5063c1fe988b 100644 --- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -59,6 +59,8 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( // Maximum number of cores to acquire (TODO: we'll need more flexible controls here) val maxCores = conf.get("spark.cores.max", Int.MaxValue.toString).toInt + val useFetcherCache = conf.getBoolean("spark.mesos.fetcherCache.enable", false) + val maxGpus = conf.getInt("spark.mesos.gpus.max", 0) private[this] val shutdownTimeoutMS = @@ -226,10 +228,10 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( s" --hostname ${offer.getHostname}" + s" --cores $numCores" + s" --app-id $appId") - command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get)) + command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get).setCache(useFetcherCache)) } - conf.getOption("spark.mesos.uris").foreach(setupUris(_, command)) + conf.getOption("spark.mesos.uris").foreach(setupUris(_, command, useFetcherCache)) command.build() } diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 73cc241239c4..9cb60237044a 100644 --- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -369,9 +369,11 @@ trait MesosSchedulerUtils extends Logging { sc.executorMemory } - def setupUris(uris: String, builder: CommandInfo.Builder): Unit = { + def setupUris(uris: String, + builder: CommandInfo.Builder, + useFetcherCache: Boolean = false): Unit = { uris.split(",").foreach { uri => - builder.addUris(CommandInfo.URI.newBuilder().setValue(uri.trim())) + builder.addUris(CommandInfo.URI.newBuilder().setValue(uri.trim()).setCache(useFetcherCache)) } } diff --git a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index 75ba02e470e2..f73638fda623 100644 --- a/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -463,6 +463,34 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite assert(launchedTasks.head.getCommand.getUrisList.asScala(0).getValue == url) } + test("mesos supports setting fetcher cache") { + val url = "spark.spark.spark.com" + setBackend(Map( + "spark.mesos.fetcherCache.enable" -> "true", + "spark.executor.uri" -> url + ), false) + val offers = List(Resources(backend.executorMemory(sc), 1)) + offerResources(offers) + val launchedTasks = verifyTaskLaunched(driver, "o1") + val uris = launchedTasks.head.getCommand.getUrisList + assert(uris.size() == 1) + assert(uris.asScala.head.getCache) + } + + test("mesos supports disabling fetcher cache") { + val url = "spark.spark.spark.com" + setBackend(Map( + "spark.mesos.fetcherCache.enable" -> "false", + "spark.executor.uri" -> url + ), false) + val offers = List(Resources(backend.executorMemory(sc), 1)) + offerResources(offers) + val launchedTasks = verifyTaskLaunched(driver, "o1") + val uris = launchedTasks.head.getCommand.getUrisList + assert(uris.size() == 1) + assert(!uris.asScala.head.getCache) + } + private case class Resources(mem: Int, cpus: Int, gpus: Int = 0) private def verifyDeclinedOffer(driver: SchedulerDriver, From ec6f479bb1d14c9eb45e0418353007be0416e4c5 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Tue, 1 Nov 2016 13:18:11 +0000 Subject: [PATCH 020/132] [SPARK-16881][MESOS] Migrate Mesos configs to use ConfigEntry ## What changes were proposed in this pull request? Migrate Mesos configs to use ConfigEntry ## How was this patch tested? Jenkins Tests Author: Sandeep Singh Closes #15654 from techaddict/SPARK-16881. --- .../deploy/mesos/MesosClusterDispatcher.scala | 9 +-- .../mesos/MesosExternalShuffleService.scala | 3 +- .../apache/spark/deploy/mesos/config.scala | 59 +++++++++++++++++++ .../deploy/mesos/ui/MesosClusterPage.scala | 3 +- 4 files changed, 68 insertions(+), 6 deletions(-) create mode 100644 mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala diff --git a/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index 73b6ca384438..7d6693b4cdf5 100644 --- a/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.mesos import java.util.concurrent.CountDownLatch import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.deploy.mesos.config._ import org.apache.spark.deploy.mesos.ui.MesosClusterUI import org.apache.spark.deploy.rest.mesos.MesosRestServer import org.apache.spark.internal.Logging @@ -51,7 +52,7 @@ private[mesos] class MesosClusterDispatcher( extends Logging { private val publicAddress = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(args.host) - private val recoveryMode = conf.get("spark.deploy.recoveryMode", "NONE").toUpperCase() + private val recoveryMode = conf.get(RECOVERY_MODE).toUpperCase() logInfo("Recovery mode in Mesos dispatcher set to: " + recoveryMode) private val engineFactory = recoveryMode match { @@ -74,7 +75,7 @@ private[mesos] class MesosClusterDispatcher( def start(): Unit = { webUi.bind() - scheduler.frameworkUrl = conf.get("spark.mesos.dispatcher.webui.url", webUi.activeWebUiUrl) + scheduler.frameworkUrl = conf.get(DISPATCHER_WEBUI_URL).getOrElse(webUi.activeWebUiUrl) scheduler.start() server.start() } @@ -99,8 +100,8 @@ private[mesos] object MesosClusterDispatcher extends Logging { conf.setMaster(dispatcherArgs.masterUrl) conf.setAppName(dispatcherArgs.name) dispatcherArgs.zookeeperUrl.foreach { z => - conf.set("spark.deploy.recoveryMode", "ZOOKEEPER") - conf.set("spark.deploy.zookeeper.url", z) + conf.set(RECOVERY_MODE, "ZOOKEEPER") + conf.set(ZOOKEEPER_URL, z) } val dispatcher = new MesosClusterDispatcher(dispatcherArgs, conf) dispatcher.start() diff --git a/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala b/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala index 6b297c4600a6..859aa836a315 100644 --- a/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala +++ b/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala @@ -24,6 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.ExternalShuffleService +import org.apache.spark.deploy.mesos.config._ import org.apache.spark.internal.Logging import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler @@ -114,7 +115,7 @@ private[mesos] class MesosExternalShuffleService(conf: SparkConf, securityManage protected override def newShuffleBlockHandler( conf: TransportConf): ExternalShuffleBlockHandler = { - val cleanerIntervalS = this.conf.getTimeAsSeconds("spark.shuffle.cleaner.interval", "30s") + val cleanerIntervalS = this.conf.get(SHUFFLE_CLEANER_INTERVAL_S) new MesosExternalShuffleBlockHandler(conf, cleanerIntervalS) } } diff --git a/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala b/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala new file mode 100644 index 000000000000..19e253394f1b --- /dev/null +++ b/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala @@ -0,0 +1,59 @@ +/* + * 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.deploy.mesos + +import java.util.concurrent.TimeUnit + +import org.apache.spark.internal.config.ConfigBuilder + +package object config { + + /* Common app configuration. */ + + private[spark] val SHUFFLE_CLEANER_INTERVAL_S = + ConfigBuilder("spark.shuffle.cleaner.interval") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("30s") + + private[spark] val RECOVERY_MODE = + ConfigBuilder("spark.deploy.recoveryMode") + .stringConf + .createWithDefault("NONE") + + private[spark] val DISPATCHER_WEBUI_URL = + ConfigBuilder("spark.mesos.dispatcher.webui.url") + .doc("Set the Spark Mesos dispatcher webui_url for interacting with the " + + "framework. If unset it will point to Spark's internal web UI.") + .stringConf + .createOptional + + private[spark] val ZOOKEEPER_URL = + ConfigBuilder("spark.deploy.zookeeper.url") + .doc("When `spark.deploy.recoveryMode` is set to ZOOKEEPER, this " + + "configuration is used to set the zookeeper URL to connect to.") + .stringConf + .createOptional + + private[spark] val HISTORY_SERVER_URL = + ConfigBuilder("spark.mesos.dispatcher.historyServer.url") + .doc("Set the URL of the history server. The dispatcher will then " + + "link each driver to its entry in the history server.") + .stringConf + .createOptional + +} diff --git a/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala b/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala index 8dcbdaad8685..13ba7d311e57 100644 --- a/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala +++ b/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala @@ -23,12 +23,13 @@ import scala.xml.Node import org.apache.mesos.Protos.TaskStatus +import org.apache.spark.deploy.mesos.config._ import org.apache.spark.deploy.mesos.MesosDriverDescription import org.apache.spark.scheduler.cluster.mesos.MesosClusterSubmissionState import org.apache.spark.ui.{UIUtils, WebUIPage} private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage("") { - private val historyServerURL = parent.conf.getOption("spark.mesos.dispatcher.historyServer.url") + private val historyServerURL = parent.conf.get(HISTORY_SERVER_URL) def render(request: HttpServletRequest): Seq[Node] = { val state = parent.scheduler.getSchedulerState() From 9b377aa49f14af31f54164378d60e0fdea2142e5 Mon Sep 17 00:00:00 2001 From: Wang Lei Date: Tue, 1 Nov 2016 13:42:10 +0000 Subject: [PATCH 021/132] [SPARK-18114][MESOS] Fix mesos cluster scheduler generage command option error ## What changes were proposed in this pull request? Enclose --conf option value with "" to support multi value configs like spark.driver.extraJavaOptions, without "", driver will fail to start. ## How was this patch tested? Jenkins Tests. Test in our production environment, also unit tests, It is a very small change. Author: Wang Lei Closes #15643 from LeightonWong/messos-cluster. --- .../spark/scheduler/cluster/mesos/MesosClusterScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 635712c00d30..8db1d126d59b 100644 --- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -482,7 +482,7 @@ private[spark] class MesosClusterScheduler( .filter { case (key, _) => !replicatedOptionsBlacklist.contains(key) } .toMap (defaultConf ++ driverConf).foreach { case (key, value) => - options ++= Seq("--conf", s"$key=${shellEscape(value)}") } + options ++= Seq("--conf", s""""$key=${shellEscape(value)}"""".stripMargin) } options } From f7c145d8ce14b23019099c509d5a2b6dfb1fe62c Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 1 Nov 2016 15:41:45 +0100 Subject: [PATCH 022/132] [SPARK-17996][SQL] Fix unqualified catalog.getFunction(...) ## What changes were proposed in this pull request? Currently an unqualified `getFunction(..)`call returns a wrong result; the returned function is shown as temporary function without a database. For example: ``` scala> sql("create function fn1 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs'") res0: org.apache.spark.sql.DataFrame = [] scala> spark.catalog.getFunction("fn1") res1: org.apache.spark.sql.catalog.Function = Function[name='fn1', className='org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs', isTemporary='true'] ``` This PR fixes this by adding database information to ExpressionInfo (which is used to store the function information). ## How was this patch tested? Added more thorough tests to `CatalogSuite`. Author: Herman van Hovell Closes #15542 from hvanhovell/SPARK-17996. --- .../sql/catalyst/expressions/ExpressionInfo.java | 14 ++++++++++++-- .../sql/catalyst/analysis/FunctionRegistry.scala | 2 +- .../sql/catalyst/catalog/SessionCatalog.scala | 10 ++++++++-- .../spark/sql/execution/command/functions.scala | 5 +++-- .../apache/spark/sql/internal/CatalogImpl.scala | 6 +++--- .../apache/spark/sql/internal/CatalogSuite.scala | 15 ++++++++++++--- 6 files changed, 39 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java index ba8e9cb4be28..4565ed44877a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java @@ -25,6 +25,7 @@ public class ExpressionInfo { private String usage; private String name; private String extended; + private String db; public String getClassName() { return className; @@ -42,14 +43,23 @@ public String getExtended() { return extended; } - public ExpressionInfo(String className, String name, String usage, String extended) { + public String getDb() { + return db; + } + + public ExpressionInfo(String className, String db, String name, String usage, String extended) { this.className = className; + this.db = db; this.name = name; this.usage = usage; this.extended = extended; } public ExpressionInfo(String className, String name) { - this(className, name, null, null); + this(className, null, name, null, null); + } + + public ExpressionInfo(String className, String db, String name) { + this(className, db, name, null, null); } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index b05f4f61f6a3..3e836ca375e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -495,7 +495,7 @@ object FunctionRegistry { val clazz = scala.reflect.classTag[T].runtimeClass val df = clazz.getAnnotation(classOf[ExpressionDescription]) if (df != null) { - new ExpressionInfo(clazz.getCanonicalName, name, df.usage(), df.extended()) + new ExpressionInfo(clazz.getCanonicalName, null, name, df.usage(), df.extended()) } else { new ExpressionInfo(clazz.getCanonicalName, name) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 3d6eec81c03c..714ef825ab83 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -943,7 +943,10 @@ class SessionCatalog( requireDbExists(db) if (externalCatalog.functionExists(db, name.funcName)) { val metadata = externalCatalog.getFunction(db, name.funcName) - new ExpressionInfo(metadata.className, qualifiedName.unquotedString) + new ExpressionInfo( + metadata.className, + qualifiedName.database.orNull, + qualifiedName.identifier) } else { failFunctionLookup(name.funcName) } @@ -1000,7 +1003,10 @@ class SessionCatalog( // catalog. So, it is possible that qualifiedName is not exactly the same as // catalogFunction.identifier.unquotedString (difference is on case-sensitivity). // At here, we preserve the input from the user. - val info = new ExpressionInfo(catalogFunction.className, qualifiedName.unquotedString) + val info = new ExpressionInfo( + catalogFunction.className, + qualifiedName.database.orNull, + qualifiedName.funcName) val builder = makeFunctionBuilder(qualifiedName.unquotedString, catalogFunction.className) createTempFunction(qualifiedName.unquotedString, info, builder, ignoreIfExists = false) // Now, we need to create the Expression. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index 26593d2918a6..24d825f5cb33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -118,14 +118,15 @@ case class DescribeFunctionCommand( case _ => try { val info = sparkSession.sessionState.catalog.lookupFunctionInfo(functionName) + val name = if (info.getDb != null) info.getDb + "." + info.getName else info.getName val result = - Row(s"Function: ${info.getName}") :: + Row(s"Function: $name") :: Row(s"Class: ${info.getClassName}") :: Row(s"Usage: ${replaceFunctionName(info.getUsage, info.getName)}") :: Nil if (isExtended) { result :+ - Row(s"Extended Usage:\n${replaceFunctionName(info.getExtended, info.getName)}") + Row(s"Extended Usage:\n${replaceFunctionName(info.getExtended, name)}") } else { result } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index f6c297e91b7c..44fd38dfb96f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -133,11 +133,11 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { private def makeFunction(funcIdent: FunctionIdentifier): Function = { val metadata = sessionCatalog.lookupFunctionInfo(funcIdent) new Function( - name = funcIdent.identifier, - database = funcIdent.database.orNull, + name = metadata.getName, + database = metadata.getDb, description = null, // for now, this is always undefined className = metadata.getClassName, - isTemporary = funcIdent.database.isEmpty) + isTemporary = metadata.getDb == null) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index 214bc736bd4d..89ec162c8ed5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -386,15 +386,24 @@ class CatalogSuite createFunction("fn2", Some(db)) // Find a temporary function - assert(spark.catalog.getFunction("fn1").name === "fn1") + val fn1 = spark.catalog.getFunction("fn1") + assert(fn1.name === "fn1") + assert(fn1.database === null) + assert(fn1.isTemporary) // Find a qualified function - assert(spark.catalog.getFunction(db, "fn2").name === "fn2") + val fn2 = spark.catalog.getFunction(db, "fn2") + assert(fn2.name === "fn2") + assert(fn2.database === db) + assert(!fn2.isTemporary) // Find an unqualified function using the current database intercept[AnalysisException](spark.catalog.getFunction("fn2")) spark.catalog.setCurrentDatabase(db) - assert(spark.catalog.getFunction("fn2").name === "fn2") + val unqualified = spark.catalog.getFunction("fn2") + assert(unqualified.name === "fn2") + assert(unqualified.database === db) + assert(!unqualified.isTemporary) } } } From 5441a6269e00e3903ae6c1ea8deb4ddf3d2e9975 Mon Sep 17 00:00:00 2001 From: eyal farago Date: Tue, 1 Nov 2016 17:12:20 +0100 Subject: [PATCH 023/132] [SPARK-16839][SQL] redundant aliases after cleanupAliases ## What changes were proposed in this pull request? Simplify struct creation, especially the aspect of `CleanupAliases` which missed some aliases when handling trees created by `CreateStruct`. This PR includes: 1. A failing test (create struct with nested aliases, some of the aliases survive `CleanupAliases`). 2. A fix that transforms `CreateStruct` into a `CreateNamedStruct` constructor, effectively eliminating `CreateStruct` from all expression trees. 3. A `NamePlaceHolder` used by `CreateStruct` when column names cannot be extracted from unresolved `NamedExpression`. 4. A new Analyzer rule that resolves `NamePlaceHolder` into a string literal once the `NamedExpression` is resolved. 5. `CleanupAliases` code was simplified as it no longer has to deal with `CreateStruct`'s top level columns. ## How was this patch tested? running all tests-suits in package org.apache.spark.sql, especially including the analysis suite, making sure added test initially fails, after applying suggested fix rerun the entire analysis package successfully. modified few tests that expected `CreateStruct` which is now transformed into `CreateNamedStruct`. Credit goes to hvanhovell for assisting with this PR. Author: eyal farago Author: eyal farago Author: Herman van Hovell Author: Eyal Farago Author: Hyukjin Kwon Author: eyalfa Closes #14444 from eyalfa/SPARK-16839_redundant_aliases_after_cleanupAliases. --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 12 +- .../sql/catalyst/analysis/Analyzer.scala | 53 ++--- .../catalyst/analysis/FunctionRegistry.scala | 2 +- .../sql/catalyst/expressions/Projection.scala | 2 - .../expressions/complexTypeCreator.scala | 211 ++++++------------ .../sql/catalyst/parser/AstBuilder.scala | 4 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 38 +++- .../expressions/ComplexTypeSuite.scala | 1 - .../scala/org/apache/spark/sql/Column.scala | 3 + .../command/AnalyzeColumnCommand.scala | 4 +- .../resources/sql-tests/inputs/group-by.sql | 2 +- .../sql-tests/results/group-by.sql.out | 4 +- .../apache/spark/sql/hive/test/TestHive.scala | 20 +- .../resources/sqlgen/subquery_in_having_2.sql | 2 +- .../sql/catalyst/LogicalPlanToSQLSuite.scala | 12 +- 15 files changed, 170 insertions(+), 200 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 9289db57b6d6..5002655fc03c 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1222,16 +1222,16 @@ test_that("column functions", { # Test struct() df <- createDataFrame(list(list(1L, 2L, 3L), list(4L, 5L, 6L)), schema = c("a", "b", "c")) - result <- collect(select(df, struct("a", "c"))) + result <- collect(select(df, alias(struct("a", "c"), "d"))) expected <- data.frame(row.names = 1:2) - expected$"struct(a, c)" <- list(listToStruct(list(a = 1L, c = 3L)), - listToStruct(list(a = 4L, c = 6L))) + expected$"d" <- list(listToStruct(list(a = 1L, c = 3L)), + listToStruct(list(a = 4L, c = 6L))) expect_equal(result, expected) - result <- collect(select(df, struct(df$a, df$b))) + result <- collect(select(df, alias(struct(df$a, df$b), "d"))) expected <- data.frame(row.names = 1:2) - expected$"struct(a, b)" <- list(listToStruct(list(a = 1L, b = 2L)), - listToStruct(list(a = 4L, b = 5L))) + expected$"d" <- list(listToStruct(list(a = 1L, b = 2L)), + listToStruct(list(a = 4L, b = 5L))) expect_equal(result, expected) # Test encode(), decode() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index f8f4799322b3..5011f2fdbf9b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, _} import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.catalyst.trees.{TreeNodeRef} +import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.toPrettySQL import org.apache.spark.sql.types._ @@ -83,6 +83,7 @@ class Analyzer( ResolveTableValuedFunctions :: ResolveRelations :: ResolveReferences :: + ResolveCreateNamedStruct :: ResolveDeserializer :: ResolveNewInstance :: ResolveUpCast :: @@ -653,11 +654,12 @@ class Analyzer( case s: Star => s.expand(child, resolver) case o => o :: Nil }) - case c: CreateStruct if containsStar(c.children) => - c.copy(children = c.children.flatMap { - case s: Star => s.expand(child, resolver) - case o => o :: Nil - }) + case c: CreateNamedStruct if containsStar(c.valExprs) => + val newChildren = c.children.grouped(2).flatMap { + case Seq(k, s : Star) => CreateStruct(s.expand(child, resolver)).children + case kv => kv + } + c.copy(children = newChildren.toList ) case c: CreateArray if containsStar(c.children) => c.copy(children = c.children.flatMap { case s: Star => s.expand(child, resolver) @@ -1141,7 +1143,7 @@ class Analyzer( case In(e, Seq(l @ ListQuery(_, exprId))) if e.resolved => // Get the left hand side expressions. val expressions = e match { - case CreateStruct(exprs) => exprs + case cns : CreateNamedStruct => cns.valExprs case expr => Seq(expr) } resolveSubQuery(l, plans, expressions.size) { (rewrite, conditions) => @@ -2072,18 +2074,8 @@ object EliminateUnions extends Rule[LogicalPlan] { */ object CleanupAliases extends Rule[LogicalPlan] { private def trimAliases(e: Expression): Expression = { - var stop = false e.transformDown { - // CreateStruct is a special case, we need to retain its top level Aliases as they decide the - // name of StructField. We also need to stop transform down this expression, or the Aliases - // under CreateStruct will be mistakenly trimmed. - case c: CreateStruct if !stop => - stop = true - c.copy(children = c.children.map(trimNonTopLevelAliases)) - case c: CreateStructUnsafe if !stop => - stop = true - c.copy(children = c.children.map(trimNonTopLevelAliases)) - case Alias(child, _) if !stop => child + case Alias(child, _) => child } } @@ -2116,15 +2108,8 @@ object CleanupAliases extends Rule[LogicalPlan] { case a: AppendColumns => a case other => - var stop = false other transformExpressionsDown { - case c: CreateStruct if !stop => - stop = true - c.copy(children = c.children.map(trimNonTopLevelAliases)) - case c: CreateStructUnsafe if !stop => - stop = true - c.copy(children = c.children.map(trimNonTopLevelAliases)) - case Alias(child, _) if !stop => child + case Alias(child, _) => child } } } @@ -2217,3 +2202,19 @@ object TimeWindowing extends Rule[LogicalPlan] { } } } + +/** + * Resolve a [[CreateNamedStruct]] if it contains [[NamePlaceholder]]s. + */ +object ResolveCreateNamedStruct extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan.transformAllExpressions { + case e: CreateNamedStruct if !e.resolved => + val children = e.children.grouped(2).flatMap { + case Seq(NamePlaceholder, e: NamedExpression) if e.resolved => + Seq(Literal(e.name), e) + case kv => + kv + } + CreateNamedStruct(children.toList) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 3e836ca375e2..b028d07fb8d0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -357,7 +357,7 @@ object FunctionRegistry { expression[MapValues]("map_values"), expression[Size]("size"), expression[SortArray]("sort_array"), - expression[CreateStruct]("struct"), + CreateStruct.registryEntry, // misc functions expression[AssertTrue]("assert_true"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index a81fa1ce3adc..03e054d09851 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -119,7 +119,6 @@ object UnsafeProjection { */ def create(exprs: Seq[Expression]): UnsafeProjection = { val unsafeExprs = exprs.map(_ transform { - case CreateStruct(children) => CreateStructUnsafe(children) case CreateNamedStruct(children) => CreateNamedStructUnsafe(children) }) GenerateUnsafeProjection.generate(unsafeExprs) @@ -145,7 +144,6 @@ object UnsafeProjection { subexpressionEliminationEnabled: Boolean): UnsafeProjection = { val e = exprs.map(BindReferences.bindReference(_, inputSchema)) .map(_ transform { - case CreateStruct(children) => CreateStructUnsafe(children) case CreateNamedStruct(children) => CreateNamedStructUnsafe(children) }) GenerateUnsafeProjection.generate(e, subexpressionEliminationEnabled) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index 917aa0873130..e9623f96e1cf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -18,9 +18,11 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.analysis.Star import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData, TypeUtils} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, TypeUtils} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -172,101 +174,70 @@ case class CreateMap(children: Seq[Expression]) extends Expression { } /** - * Returns a Row containing the evaluation of all children expressions. + * An expression representing a not yet available attribute name. This expression is unevaluable + * and as its name suggests it is a temporary place holder until we're able to determine the + * actual attribute name. */ -@ExpressionDescription( - usage = "_FUNC_(col1, col2, col3, ...) - Creates a struct with the given field values.") -case class CreateStruct(children: Seq[Expression]) extends Expression { - - override def foldable: Boolean = children.forall(_.foldable) - - override lazy val dataType: StructType = { - val fields = children.zipWithIndex.map { case (child, idx) => - child match { - case ne: NamedExpression => - StructField(ne.name, ne.dataType, ne.nullable, ne.metadata) - case _ => - StructField(s"col${idx + 1}", child.dataType, child.nullable, Metadata.empty) - } - } - StructType(fields) - } - +case object NamePlaceholder extends LeafExpression with Unevaluable { + override lazy val resolved: Boolean = false + override def foldable: Boolean = false override def nullable: Boolean = false + override def dataType: DataType = StringType + override def prettyName: String = "NamePlaceholder" + override def toString: String = prettyName +} - override def eval(input: InternalRow): Any = { - InternalRow(children.map(_.eval(input)): _*) +/** + * Returns a Row containing the evaluation of all children expressions. + */ +object CreateStruct extends FunctionBuilder { + def apply(children: Seq[Expression]): CreateNamedStruct = { + CreateNamedStruct(children.zipWithIndex.flatMap { + case (e: NamedExpression, _) if e.resolved => Seq(Literal(e.name), e) + case (e: NamedExpression, _) => Seq(NamePlaceholder, e) + case (e, index) => Seq(Literal(s"col${index + 1}"), e) + }) } - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val rowClass = classOf[GenericInternalRow].getName - val values = ctx.freshName("values") - ctx.addMutableState("Object[]", values, s"this.$values = null;") - - ev.copy(code = s""" - boolean ${ev.isNull} = false; - this.$values = new Object[${children.size}];""" + - ctx.splitExpressions( - ctx.INPUT_ROW, - children.zipWithIndex.map { case (e, i) => - val eval = e.genCode(ctx) - eval.code + s""" - if (${eval.isNull}) { - $values[$i] = null; - } else { - $values[$i] = ${eval.value}; - }""" - }) + - s""" - final InternalRow ${ev.value} = new $rowClass($values); - this.$values = null; - """) + /** + * Entry to use in the function registry. + */ + val registryEntry: (String, (ExpressionInfo, FunctionBuilder)) = { + val info: ExpressionInfo = new ExpressionInfo( + "org.apache.spark.sql.catalyst.expressions.NamedStruct", + "struct", + "_FUNC_(col1, col2, col3, ...) - Creates a struct with the given field values.", + "") + ("struct", (info, this)) } - - override def prettyName: String = "struct" } - /** - * Creates a struct with the given field names and values - * - * @param children Seq(name1, val1, name2, val2, ...) + * Common base class for both [[CreateNamedStruct]] and [[CreateNamedStructUnsafe]]. */ -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values.") -// scalastyle:on line.size.limit -case class CreateNamedStruct(children: Seq[Expression]) extends Expression { +trait CreateNamedStructLike extends Expression { + lazy val (nameExprs, valExprs) = children.grouped(2).map { + case Seq(name, value) => (name, value) + }.toList.unzip - /** - * Returns Aliased [[Expression]]s that could be used to construct a flattened version of this - * StructType. - */ - def flatten: Seq[NamedExpression] = valExprs.zip(names).map { - case (v, n) => Alias(v, n.toString)() - } + lazy val names = nameExprs.map(_.eval(EmptyRow)) - private lazy val (nameExprs, valExprs) = - children.grouped(2).map { case Seq(name, value) => (name, value) }.toList.unzip + override def nullable: Boolean = false - private lazy val names = nameExprs.map(_.eval(EmptyRow)) + override def foldable: Boolean = valExprs.forall(_.foldable) override lazy val dataType: StructType = { val fields = names.zip(valExprs).map { - case (name, valExpr: NamedExpression) => - StructField(name.asInstanceOf[UTF8String].toString, - valExpr.dataType, valExpr.nullable, valExpr.metadata) - case (name, valExpr) => - StructField(name.asInstanceOf[UTF8String].toString, - valExpr.dataType, valExpr.nullable, Metadata.empty) + case (name, expr) => + val metadata = expr match { + case ne: NamedExpression => ne.metadata + case _ => Metadata.empty + } + StructField(name.toString, expr.dataType, expr.nullable, metadata) } StructType(fields) } - override def foldable: Boolean = valExprs.forall(_.foldable) - - override def nullable: Boolean = false - override def checkInputDataTypes(): TypeCheckResult = { if (children.size % 2 != 0) { TypeCheckResult.TypeCheckFailure(s"$prettyName expects an even number of arguments.") @@ -274,8 +245,8 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression { val invalidNames = nameExprs.filterNot(e => e.foldable && e.dataType == StringType) if (invalidNames.nonEmpty) { TypeCheckResult.TypeCheckFailure( - s"Only foldable StringType expressions are allowed to appear at odd position , got :" + - s" ${invalidNames.mkString(",")}") + "Only foldable StringType expressions are allowed to appear at odd position, got:" + + s" ${invalidNames.mkString(",")}") } else if (!names.contains(null)) { TypeCheckResult.TypeCheckSuccess } else { @@ -284,9 +255,29 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression { } } + /** + * Returns Aliased [[Expression]]s that could be used to construct a flattened version of this + * StructType. + */ + def flatten: Seq[NamedExpression] = valExprs.zip(names).map { + case (v, n) => Alias(v, n.toString)() + } + override def eval(input: InternalRow): Any = { InternalRow(valExprs.map(_.eval(input)): _*) } +} + +/** + * Creates a struct with the given field names and values + * + * @param children Seq(name1, val1, name2, val2, ...) + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values.") +// scalastyle:on line.size.limit +case class CreateNamedStruct(children: Seq[Expression]) extends CreateNamedStructLike { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val rowClass = classOf[GenericInternalRow].getName @@ -316,44 +307,6 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression { override def prettyName: String = "named_struct" } -/** - * Returns a Row containing the evaluation of all children expressions. This is a variant that - * returns UnsafeRow directly. The unsafe projection operator replaces [[CreateStruct]] with - * this expression automatically at runtime. - */ -case class CreateStructUnsafe(children: Seq[Expression]) extends Expression { - - override def foldable: Boolean = children.forall(_.foldable) - - override lazy val resolved: Boolean = childrenResolved - - override lazy val dataType: StructType = { - val fields = children.zipWithIndex.map { case (child, idx) => - child match { - case ne: NamedExpression => - StructField(ne.name, ne.dataType, ne.nullable, ne.metadata) - case _ => - StructField(s"col${idx + 1}", child.dataType, child.nullable, Metadata.empty) - } - } - StructType(fields) - } - - override def nullable: Boolean = false - - override def eval(input: InternalRow): Any = { - InternalRow(children.map(_.eval(input)): _*) - } - - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val eval = GenerateUnsafeProjection.createCode(ctx, children) - ExprCode(code = eval.code, isNull = eval.isNull, value = eval.value) - } - - override def prettyName: String = "struct_unsafe" -} - - /** * Creates a struct with the given field names and values. This is a variant that returns * UnsafeRow directly. The unsafe projection operator replaces [[CreateStruct]] with @@ -361,31 +314,7 @@ case class CreateStructUnsafe(children: Seq[Expression]) extends Expression { * * @param children Seq(name1, val1, name2, val2, ...) */ -case class CreateNamedStructUnsafe(children: Seq[Expression]) extends Expression { - - private lazy val (nameExprs, valExprs) = - children.grouped(2).map { case Seq(name, value) => (name, value) }.toList.unzip - - private lazy val names = nameExprs.map(_.eval(EmptyRow).toString) - - override lazy val dataType: StructType = { - val fields = names.zip(valExprs).map { - case (name, valExpr: NamedExpression) => - StructField(name, valExpr.dataType, valExpr.nullable, valExpr.metadata) - case (name, valExpr) => - StructField(name, valExpr.dataType, valExpr.nullable, Metadata.empty) - } - StructType(fields) - } - - override def foldable: Boolean = valExprs.forall(_.foldable) - - override def nullable: Boolean = false - - override def eval(input: InternalRow): Any = { - InternalRow(valExprs.map(_.eval(input)): _*) - } - +case class CreateNamedStructUnsafe(children: Seq[Expression]) extends CreateNamedStructLike { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val eval = GenerateUnsafeProjection.createCode(ctx, valExprs) ExprCode(code = eval.code, isNull = eval.isNull, value = eval.value) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 38e9bb6c162a..35aca91cf882 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -681,8 +681,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { // inline table comes in two styles: // style 1: values (1), (2), (3) -- multiple columns are supported // style 2: values 1, 2, 3 -- only a single column is supported here - case CreateStruct(children) => children // style 1 - case child => Seq(child) // style 2 + case struct: CreateNamedStruct => struct.valExprs // style 1 + case child => Seq(child) // style 2 } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 590774c04304..817de48de279 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import org.scalatest.ShouldMatchers + import org.apache.spark.sql.catalyst.{SimpleCatalystConf, TableIdentifier} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -25,7 +27,8 @@ import org.apache.spark.sql.catalyst.plans.{Cross, Inner} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types._ -class AnalysisSuite extends AnalysisTest { + +class AnalysisSuite extends AnalysisTest with ShouldMatchers { import org.apache.spark.sql.catalyst.analysis.TestRelations._ test("union project *") { @@ -218,9 +221,36 @@ class AnalysisSuite extends AnalysisTest { // CreateStruct is a special case that we should not trim Alias for it. plan = testRelation.select(CreateStruct(Seq(a, (a + 1).as("a+1"))).as("col")) - checkAnalysis(plan, plan) - plan = testRelation.select(CreateStructUnsafe(Seq(a, (a + 1).as("a+1"))).as("col")) - checkAnalysis(plan, plan) + expected = testRelation.select(CreateNamedStruct(Seq( + Literal(a.name), a, + Literal("a+1"), (a + 1))).as("col")) + checkAnalysis(plan, expected) + } + + test("Analysis may leave unnecassary aliases") { + val att1 = testRelation.output.head + var plan = testRelation.select( + CreateStruct(Seq(att1, ((att1.as("aa")) + 1).as("a_plus_1"))).as("col"), + att1 + ) + val prevPlan = getAnalyzer(true).execute(plan) + plan = prevPlan.select(CreateArray(Seq( + CreateStruct(Seq(att1, (att1 + 1).as("a_plus_1"))).as("col1"), + /** alias should be eliminated by [[CleanupAliases]] */ + "col".attr.as("col2") + )).as("arr")) + plan = getAnalyzer(true).execute(plan) + + val expectedPlan = prevPlan.select( + CreateArray(Seq( + CreateNamedStruct(Seq( + Literal(att1.name), att1, + Literal("a_plus_1"), (att1 + 1))), + 'col.struct(prevPlan.output(0).dataType.asInstanceOf[StructType]).notNull + )).as("arr") + ) + + checkAnalysis(plan, expectedPlan) } test("SPARK-10534: resolve attribute references in order by clause") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala index 0c307b2b8576..c21c6de32c0b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala @@ -243,7 +243,6 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { val b = AttributeReference("b", IntegerType)() checkMetadata(CreateStruct(Seq(a, b))) checkMetadata(CreateNamedStruct(Seq("a", a, "b", b))) - checkMetadata(CreateStructUnsafe(Seq(a, b))) checkMetadata(CreateNamedStructUnsafe(Seq("a", a, "b", b))) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 05e867bf5be9..067b0bac6303 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -183,6 +183,9 @@ class Column(protected[sql] val expr: Expression) extends Logging { case a: AggregateExpression if a.aggregateFunction.isInstanceOf[TypedAggregateExpression] => UnresolvedAlias(a, Some(Column.generateAlias)) + // Wait until the struct is resolved. This will generate a nicer looking alias. + case struct: CreateNamedStructLike => UnresolvedAlias(struct) + case expr: Expression => Alias(expr, usePrettyExpression(expr).sql)() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala index f873f34a845e..6141fab4aff0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala @@ -137,7 +137,7 @@ object ColumnStatStruct { private def numTrues(e: Expression): Expression = Sum(If(e, one, zero)) private def numFalses(e: Expression): Expression = Sum(If(Not(e), one, zero)) - private def getStruct(exprs: Seq[Expression]): CreateStruct = { + private def getStruct(exprs: Seq[Expression]): CreateNamedStruct = { CreateStruct(exprs.map { expr: Expression => expr.transformUp { case af: AggregateFunction => af.toAggregateExpression() @@ -168,7 +168,7 @@ object ColumnStatStruct { } } - def apply(attr: Attribute, relativeSD: Double): CreateStruct = attr.dataType match { + def apply(attr: Attribute, relativeSD: Double): CreateNamedStruct = attr.dataType match { // Use aggregate functions to compute statistics we need. case _: NumericType | TimestampType | DateType => getStruct(numericColumnStat(attr, relativeSD)) case StringType => getStruct(stringColumnStat(attr, relativeSD)) diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index 6741703d9d82..d496af686d75 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -14,4 +14,4 @@ select 'foo' from myview where int_col == 0 group by 1; select 'foo', approx_count_distinct(int_col) from myview where int_col == 0 group by 1; -- group-by should not produce any rows (sort aggregate). -select 'foo', max(struct(int_col)) from myview where int_col == 0 group by 1; +select 'foo', max(struct(int_col)) as agg_struct from myview where int_col == 0 group by 1; diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 9127bd4dd4c6..dede3a09ce75 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -44,8 +44,8 @@ struct -- !query 5 -select 'foo', max(struct(int_col)) from myview where int_col == 0 group by 1 +select 'foo', max(struct(int_col)) as agg_struct from myview where int_col == 0 group by 1 -- !query 5 schema -struct> +struct> -- !query 5 output diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 6eb571b91ffa..90000445dffb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -190,6 +190,12 @@ private[hive] class TestHiveSparkSession( new File(Thread.currentThread().getContextClassLoader.getResource(path).getFile) } + private def quoteHiveFile(path : String) = if (Utils.isWindows) { + getHiveFile(path).getPath.replace('\\', '/') + } else { + getHiveFile(path).getPath + } + def getWarehousePath(): String = { val tempConf = new SQLConf sc.conf.getAll.foreach { case (k, v) => tempConf.setConfString(k, v) } @@ -225,16 +231,16 @@ private[hive] class TestHiveSparkSession( val hiveQTestUtilTables: Seq[TestTable] = Seq( TestTable("src", "CREATE TABLE src (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' INTO TABLE src".cmd), + s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' INTO TABLE src".cmd), TestTable("src1", "CREATE TABLE src1 (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), + s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), TestTable("srcpart", () => { sql( "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { sql( - s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' + s"""LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') """.stripMargin) } @@ -244,7 +250,7 @@ private[hive] class TestHiveSparkSession( "CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { sql( - s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' + s"""LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') """.stripMargin) } @@ -269,7 +275,7 @@ private[hive] class TestHiveSparkSession( sql( s""" - |LOAD DATA LOCAL INPATH '${getHiveFile("data/files/complex.seq")}' + |LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/complex.seq")}' |INTO TABLE src_thrift """.stripMargin) }), @@ -308,7 +314,7 @@ private[hive] class TestHiveSparkSession( |) """.stripMargin.cmd, s""" - |LOAD DATA LOCAL INPATH '${getHiveFile("data/files/episodes.avro")}' + |LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/episodes.avro")}' |INTO TABLE episodes """.stripMargin.cmd ), @@ -379,7 +385,7 @@ private[hive] class TestHiveSparkSession( TestTable("src_json", s"""CREATE TABLE src_json (json STRING) STORED AS TEXTFILE """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/json.txt")}' INTO TABLE src_json".cmd) + s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/json.txt")}' INTO TABLE src_json".cmd) ) hiveQTestUtilTables.foreach(registerTestTable) diff --git a/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql index de0116a4dcba..cdda29af50e3 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql @@ -7,4 +7,4 @@ having b.key in (select a.key where a.value > 'val_9' and a.value = min(b.value)) order by b.key -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_5`) AS `gen_attr_1`, min(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (struct(`gen_attr_0`, `gen_attr_4`) IN (SELECT `gen_attr_6` AS `_c0`, `gen_attr_7` AS `_c1` FROM (SELECT `gen_attr_2` AS `gen_attr_6`, `gen_attr_3` AS `gen_attr_7` FROM (SELECT `gen_attr_2`, `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_3` > 'val_9')) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC NULLS FIRST) AS b +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_5`) AS `gen_attr_1`, min(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (named_struct('gen_attr_0', `gen_attr_0`, 'gen_attr_4', `gen_attr_4`) IN (SELECT `gen_attr_6` AS `_c0`, `gen_attr_7` AS `_c1` FROM (SELECT `gen_attr_2` AS `gen_attr_6`, `gen_attr_3` AS `gen_attr_7` FROM (SELECT `gen_attr_2`, `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_3` > 'val_9')) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC NULLS FIRST) AS b diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index c7f10e569fa4..12d18dc87ceb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst import java.nio.charset.StandardCharsets import java.nio.file.{Files, NoSuchFileException, Paths} +import scala.io.Source import scala.util.control.NonFatal import org.apache.spark.sql.Column @@ -109,12 +110,15 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { Files.write(path, answerText.getBytes(StandardCharsets.UTF_8)) } else { val goldenFileName = s"sqlgen/$answerFile.sql" - val resourceFile = getClass.getClassLoader.getResource(goldenFileName) - if (resourceFile == null) { + val resourceStream = getClass.getClassLoader.getResourceAsStream(goldenFileName) + if (resourceStream == null) { throw new NoSuchFileException(goldenFileName) } - val path = resourceFile.getPath - val answerText = new String(Files.readAllBytes(Paths.get(path)), StandardCharsets.UTF_8) + val answerText = try { + Source.fromInputStream(resourceStream).mkString + } finally { + resourceStream.close + } val sqls = answerText.split(separator) assert(sqls.length == 2, "Golden sql files should have a separator.") val expectedSQL = sqls(1).trim() From 0cba535af3c65618f342fa2d7db9647f5e6f6f1b Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Tue, 1 Nov 2016 17:30:37 +0100 Subject: [PATCH 024/132] Revert "[SPARK-16839][SQL] redundant aliases after cleanupAliases" This reverts commit 5441a6269e00e3903ae6c1ea8deb4ddf3d2e9975. --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 12 +- .../sql/catalyst/analysis/Analyzer.scala | 53 +++-- .../catalyst/analysis/FunctionRegistry.scala | 2 +- .../sql/catalyst/expressions/Projection.scala | 2 + .../expressions/complexTypeCreator.scala | 211 ++++++++++++------ .../sql/catalyst/parser/AstBuilder.scala | 4 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 38 +--- .../expressions/ComplexTypeSuite.scala | 1 + .../scala/org/apache/spark/sql/Column.scala | 3 - .../command/AnalyzeColumnCommand.scala | 4 +- .../resources/sql-tests/inputs/group-by.sql | 2 +- .../sql-tests/results/group-by.sql.out | 4 +- .../apache/spark/sql/hive/test/TestHive.scala | 20 +- .../resources/sqlgen/subquery_in_having_2.sql | 2 +- .../sql/catalyst/LogicalPlanToSQLSuite.scala | 12 +- 15 files changed, 200 insertions(+), 170 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 5002655fc03c..9289db57b6d6 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1222,16 +1222,16 @@ test_that("column functions", { # Test struct() df <- createDataFrame(list(list(1L, 2L, 3L), list(4L, 5L, 6L)), schema = c("a", "b", "c")) - result <- collect(select(df, alias(struct("a", "c"), "d"))) + result <- collect(select(df, struct("a", "c"))) expected <- data.frame(row.names = 1:2) - expected$"d" <- list(listToStruct(list(a = 1L, c = 3L)), - listToStruct(list(a = 4L, c = 6L))) + expected$"struct(a, c)" <- list(listToStruct(list(a = 1L, c = 3L)), + listToStruct(list(a = 4L, c = 6L))) expect_equal(result, expected) - result <- collect(select(df, alias(struct(df$a, df$b), "d"))) + result <- collect(select(df, struct(df$a, df$b))) expected <- data.frame(row.names = 1:2) - expected$"d" <- list(listToStruct(list(a = 1L, b = 2L)), - listToStruct(list(a = 4L, b = 5L))) + expected$"struct(a, b)" <- list(listToStruct(list(a = 1L, b = 2L)), + listToStruct(list(a = 4L, b = 5L))) expect_equal(result, expected) # Test encode(), decode() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 5011f2fdbf9b..f8f4799322b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, _} import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.catalyst.trees.TreeNodeRef +import org.apache.spark.sql.catalyst.trees.{TreeNodeRef} import org.apache.spark.sql.catalyst.util.toPrettySQL import org.apache.spark.sql.types._ @@ -83,7 +83,6 @@ class Analyzer( ResolveTableValuedFunctions :: ResolveRelations :: ResolveReferences :: - ResolveCreateNamedStruct :: ResolveDeserializer :: ResolveNewInstance :: ResolveUpCast :: @@ -654,12 +653,11 @@ class Analyzer( case s: Star => s.expand(child, resolver) case o => o :: Nil }) - case c: CreateNamedStruct if containsStar(c.valExprs) => - val newChildren = c.children.grouped(2).flatMap { - case Seq(k, s : Star) => CreateStruct(s.expand(child, resolver)).children - case kv => kv - } - c.copy(children = newChildren.toList ) + case c: CreateStruct if containsStar(c.children) => + c.copy(children = c.children.flatMap { + case s: Star => s.expand(child, resolver) + case o => o :: Nil + }) case c: CreateArray if containsStar(c.children) => c.copy(children = c.children.flatMap { case s: Star => s.expand(child, resolver) @@ -1143,7 +1141,7 @@ class Analyzer( case In(e, Seq(l @ ListQuery(_, exprId))) if e.resolved => // Get the left hand side expressions. val expressions = e match { - case cns : CreateNamedStruct => cns.valExprs + case CreateStruct(exprs) => exprs case expr => Seq(expr) } resolveSubQuery(l, plans, expressions.size) { (rewrite, conditions) => @@ -2074,8 +2072,18 @@ object EliminateUnions extends Rule[LogicalPlan] { */ object CleanupAliases extends Rule[LogicalPlan] { private def trimAliases(e: Expression): Expression = { + var stop = false e.transformDown { - case Alias(child, _) => child + // CreateStruct is a special case, we need to retain its top level Aliases as they decide the + // name of StructField. We also need to stop transform down this expression, or the Aliases + // under CreateStruct will be mistakenly trimmed. + case c: CreateStruct if !stop => + stop = true + c.copy(children = c.children.map(trimNonTopLevelAliases)) + case c: CreateStructUnsafe if !stop => + stop = true + c.copy(children = c.children.map(trimNonTopLevelAliases)) + case Alias(child, _) if !stop => child } } @@ -2108,8 +2116,15 @@ object CleanupAliases extends Rule[LogicalPlan] { case a: AppendColumns => a case other => + var stop = false other transformExpressionsDown { - case Alias(child, _) => child + case c: CreateStruct if !stop => + stop = true + c.copy(children = c.children.map(trimNonTopLevelAliases)) + case c: CreateStructUnsafe if !stop => + stop = true + c.copy(children = c.children.map(trimNonTopLevelAliases)) + case Alias(child, _) if !stop => child } } } @@ -2202,19 +2217,3 @@ object TimeWindowing extends Rule[LogicalPlan] { } } } - -/** - * Resolve a [[CreateNamedStruct]] if it contains [[NamePlaceholder]]s. - */ -object ResolveCreateNamedStruct extends Rule[LogicalPlan] { - override def apply(plan: LogicalPlan): LogicalPlan = plan.transformAllExpressions { - case e: CreateNamedStruct if !e.resolved => - val children = e.children.grouped(2).flatMap { - case Seq(NamePlaceholder, e: NamedExpression) if e.resolved => - Seq(Literal(e.name), e) - case kv => - kv - } - CreateNamedStruct(children.toList) - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index b028d07fb8d0..3e836ca375e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -357,7 +357,7 @@ object FunctionRegistry { expression[MapValues]("map_values"), expression[Size]("size"), expression[SortArray]("sort_array"), - CreateStruct.registryEntry, + expression[CreateStruct]("struct"), // misc functions expression[AssertTrue]("assert_true"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 03e054d09851..a81fa1ce3adc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -119,6 +119,7 @@ object UnsafeProjection { */ def create(exprs: Seq[Expression]): UnsafeProjection = { val unsafeExprs = exprs.map(_ transform { + case CreateStruct(children) => CreateStructUnsafe(children) case CreateNamedStruct(children) => CreateNamedStructUnsafe(children) }) GenerateUnsafeProjection.generate(unsafeExprs) @@ -144,6 +145,7 @@ object UnsafeProjection { subexpressionEliminationEnabled: Boolean): UnsafeProjection = { val e = exprs.map(BindReferences.bindReference(_, inputSchema)) .map(_ transform { + case CreateStruct(children) => CreateStructUnsafe(children) case CreateNamedStruct(children) => CreateNamedStructUnsafe(children) }) GenerateUnsafeProjection.generate(e, subexpressionEliminationEnabled) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index e9623f96e1cf..917aa0873130 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -18,11 +18,9 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder -import org.apache.spark.sql.catalyst.analysis.Star import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, TypeUtils} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData, TypeUtils} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -174,70 +172,101 @@ case class CreateMap(children: Seq[Expression]) extends Expression { } /** - * An expression representing a not yet available attribute name. This expression is unevaluable - * and as its name suggests it is a temporary place holder until we're able to determine the - * actual attribute name. + * Returns a Row containing the evaluation of all children expressions. */ -case object NamePlaceholder extends LeafExpression with Unevaluable { - override lazy val resolved: Boolean = false - override def foldable: Boolean = false +@ExpressionDescription( + usage = "_FUNC_(col1, col2, col3, ...) - Creates a struct with the given field values.") +case class CreateStruct(children: Seq[Expression]) extends Expression { + + override def foldable: Boolean = children.forall(_.foldable) + + override lazy val dataType: StructType = { + val fields = children.zipWithIndex.map { case (child, idx) => + child match { + case ne: NamedExpression => + StructField(ne.name, ne.dataType, ne.nullable, ne.metadata) + case _ => + StructField(s"col${idx + 1}", child.dataType, child.nullable, Metadata.empty) + } + } + StructType(fields) + } + override def nullable: Boolean = false - override def dataType: DataType = StringType - override def prettyName: String = "NamePlaceholder" - override def toString: String = prettyName -} -/** - * Returns a Row containing the evaluation of all children expressions. - */ -object CreateStruct extends FunctionBuilder { - def apply(children: Seq[Expression]): CreateNamedStruct = { - CreateNamedStruct(children.zipWithIndex.flatMap { - case (e: NamedExpression, _) if e.resolved => Seq(Literal(e.name), e) - case (e: NamedExpression, _) => Seq(NamePlaceholder, e) - case (e, index) => Seq(Literal(s"col${index + 1}"), e) - }) + override def eval(input: InternalRow): Any = { + InternalRow(children.map(_.eval(input)): _*) } - /** - * Entry to use in the function registry. - */ - val registryEntry: (String, (ExpressionInfo, FunctionBuilder)) = { - val info: ExpressionInfo = new ExpressionInfo( - "org.apache.spark.sql.catalyst.expressions.NamedStruct", - "struct", - "_FUNC_(col1, col2, col3, ...) - Creates a struct with the given field values.", - "") - ("struct", (info, this)) + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val rowClass = classOf[GenericInternalRow].getName + val values = ctx.freshName("values") + ctx.addMutableState("Object[]", values, s"this.$values = null;") + + ev.copy(code = s""" + boolean ${ev.isNull} = false; + this.$values = new Object[${children.size}];""" + + ctx.splitExpressions( + ctx.INPUT_ROW, + children.zipWithIndex.map { case (e, i) => + val eval = e.genCode(ctx) + eval.code + s""" + if (${eval.isNull}) { + $values[$i] = null; + } else { + $values[$i] = ${eval.value}; + }""" + }) + + s""" + final InternalRow ${ev.value} = new $rowClass($values); + this.$values = null; + """) } + + override def prettyName: String = "struct" } + /** - * Common base class for both [[CreateNamedStruct]] and [[CreateNamedStructUnsafe]]. + * Creates a struct with the given field names and values + * + * @param children Seq(name1, val1, name2, val2, ...) */ -trait CreateNamedStructLike extends Expression { - lazy val (nameExprs, valExprs) = children.grouped(2).map { - case Seq(name, value) => (name, value) - }.toList.unzip +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values.") +// scalastyle:on line.size.limit +case class CreateNamedStruct(children: Seq[Expression]) extends Expression { - lazy val names = nameExprs.map(_.eval(EmptyRow)) + /** + * Returns Aliased [[Expression]]s that could be used to construct a flattened version of this + * StructType. + */ + def flatten: Seq[NamedExpression] = valExprs.zip(names).map { + case (v, n) => Alias(v, n.toString)() + } - override def nullable: Boolean = false + private lazy val (nameExprs, valExprs) = + children.grouped(2).map { case Seq(name, value) => (name, value) }.toList.unzip - override def foldable: Boolean = valExprs.forall(_.foldable) + private lazy val names = nameExprs.map(_.eval(EmptyRow)) override lazy val dataType: StructType = { val fields = names.zip(valExprs).map { - case (name, expr) => - val metadata = expr match { - case ne: NamedExpression => ne.metadata - case _ => Metadata.empty - } - StructField(name.toString, expr.dataType, expr.nullable, metadata) + case (name, valExpr: NamedExpression) => + StructField(name.asInstanceOf[UTF8String].toString, + valExpr.dataType, valExpr.nullable, valExpr.metadata) + case (name, valExpr) => + StructField(name.asInstanceOf[UTF8String].toString, + valExpr.dataType, valExpr.nullable, Metadata.empty) } StructType(fields) } + override def foldable: Boolean = valExprs.forall(_.foldable) + + override def nullable: Boolean = false + override def checkInputDataTypes(): TypeCheckResult = { if (children.size % 2 != 0) { TypeCheckResult.TypeCheckFailure(s"$prettyName expects an even number of arguments.") @@ -245,8 +274,8 @@ trait CreateNamedStructLike extends Expression { val invalidNames = nameExprs.filterNot(e => e.foldable && e.dataType == StringType) if (invalidNames.nonEmpty) { TypeCheckResult.TypeCheckFailure( - "Only foldable StringType expressions are allowed to appear at odd position, got:" + - s" ${invalidNames.mkString(",")}") + s"Only foldable StringType expressions are allowed to appear at odd position , got :" + + s" ${invalidNames.mkString(",")}") } else if (!names.contains(null)) { TypeCheckResult.TypeCheckSuccess } else { @@ -255,29 +284,9 @@ trait CreateNamedStructLike extends Expression { } } - /** - * Returns Aliased [[Expression]]s that could be used to construct a flattened version of this - * StructType. - */ - def flatten: Seq[NamedExpression] = valExprs.zip(names).map { - case (v, n) => Alias(v, n.toString)() - } - override def eval(input: InternalRow): Any = { InternalRow(valExprs.map(_.eval(input)): _*) } -} - -/** - * Creates a struct with the given field names and values - * - * @param children Seq(name1, val1, name2, val2, ...) - */ -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values.") -// scalastyle:on line.size.limit -case class CreateNamedStruct(children: Seq[Expression]) extends CreateNamedStructLike { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val rowClass = classOf[GenericInternalRow].getName @@ -307,6 +316,44 @@ case class CreateNamedStruct(children: Seq[Expression]) extends CreateNamedStruc override def prettyName: String = "named_struct" } +/** + * Returns a Row containing the evaluation of all children expressions. This is a variant that + * returns UnsafeRow directly. The unsafe projection operator replaces [[CreateStruct]] with + * this expression automatically at runtime. + */ +case class CreateStructUnsafe(children: Seq[Expression]) extends Expression { + + override def foldable: Boolean = children.forall(_.foldable) + + override lazy val resolved: Boolean = childrenResolved + + override lazy val dataType: StructType = { + val fields = children.zipWithIndex.map { case (child, idx) => + child match { + case ne: NamedExpression => + StructField(ne.name, ne.dataType, ne.nullable, ne.metadata) + case _ => + StructField(s"col${idx + 1}", child.dataType, child.nullable, Metadata.empty) + } + } + StructType(fields) + } + + override def nullable: Boolean = false + + override def eval(input: InternalRow): Any = { + InternalRow(children.map(_.eval(input)): _*) + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val eval = GenerateUnsafeProjection.createCode(ctx, children) + ExprCode(code = eval.code, isNull = eval.isNull, value = eval.value) + } + + override def prettyName: String = "struct_unsafe" +} + + /** * Creates a struct with the given field names and values. This is a variant that returns * UnsafeRow directly. The unsafe projection operator replaces [[CreateStruct]] with @@ -314,7 +361,31 @@ case class CreateNamedStruct(children: Seq[Expression]) extends CreateNamedStruc * * @param children Seq(name1, val1, name2, val2, ...) */ -case class CreateNamedStructUnsafe(children: Seq[Expression]) extends CreateNamedStructLike { +case class CreateNamedStructUnsafe(children: Seq[Expression]) extends Expression { + + private lazy val (nameExprs, valExprs) = + children.grouped(2).map { case Seq(name, value) => (name, value) }.toList.unzip + + private lazy val names = nameExprs.map(_.eval(EmptyRow).toString) + + override lazy val dataType: StructType = { + val fields = names.zip(valExprs).map { + case (name, valExpr: NamedExpression) => + StructField(name, valExpr.dataType, valExpr.nullable, valExpr.metadata) + case (name, valExpr) => + StructField(name, valExpr.dataType, valExpr.nullable, Metadata.empty) + } + StructType(fields) + } + + override def foldable: Boolean = valExprs.forall(_.foldable) + + override def nullable: Boolean = false + + override def eval(input: InternalRow): Any = { + InternalRow(valExprs.map(_.eval(input)): _*) + } + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val eval = GenerateUnsafeProjection.createCode(ctx, valExprs) ExprCode(code = eval.code, isNull = eval.isNull, value = eval.value) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 35aca91cf882..38e9bb6c162a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -681,8 +681,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { // inline table comes in two styles: // style 1: values (1), (2), (3) -- multiple columns are supported // style 2: values 1, 2, 3 -- only a single column is supported here - case struct: CreateNamedStruct => struct.valExprs // style 1 - case child => Seq(child) // style 2 + case CreateStruct(children) => children // style 1 + case child => Seq(child) // style 2 } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 817de48de279..590774c04304 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.analysis -import org.scalatest.ShouldMatchers - import org.apache.spark.sql.catalyst.{SimpleCatalystConf, TableIdentifier} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -27,8 +25,7 @@ import org.apache.spark.sql.catalyst.plans.{Cross, Inner} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types._ - -class AnalysisSuite extends AnalysisTest with ShouldMatchers { +class AnalysisSuite extends AnalysisTest { import org.apache.spark.sql.catalyst.analysis.TestRelations._ test("union project *") { @@ -221,36 +218,9 @@ class AnalysisSuite extends AnalysisTest with ShouldMatchers { // CreateStruct is a special case that we should not trim Alias for it. plan = testRelation.select(CreateStruct(Seq(a, (a + 1).as("a+1"))).as("col")) - expected = testRelation.select(CreateNamedStruct(Seq( - Literal(a.name), a, - Literal("a+1"), (a + 1))).as("col")) - checkAnalysis(plan, expected) - } - - test("Analysis may leave unnecassary aliases") { - val att1 = testRelation.output.head - var plan = testRelation.select( - CreateStruct(Seq(att1, ((att1.as("aa")) + 1).as("a_plus_1"))).as("col"), - att1 - ) - val prevPlan = getAnalyzer(true).execute(plan) - plan = prevPlan.select(CreateArray(Seq( - CreateStruct(Seq(att1, (att1 + 1).as("a_plus_1"))).as("col1"), - /** alias should be eliminated by [[CleanupAliases]] */ - "col".attr.as("col2") - )).as("arr")) - plan = getAnalyzer(true).execute(plan) - - val expectedPlan = prevPlan.select( - CreateArray(Seq( - CreateNamedStruct(Seq( - Literal(att1.name), att1, - Literal("a_plus_1"), (att1 + 1))), - 'col.struct(prevPlan.output(0).dataType.asInstanceOf[StructType]).notNull - )).as("arr") - ) - - checkAnalysis(plan, expectedPlan) + checkAnalysis(plan, plan) + plan = testRelation.select(CreateStructUnsafe(Seq(a, (a + 1).as("a+1"))).as("col")) + checkAnalysis(plan, plan) } test("SPARK-10534: resolve attribute references in order by clause") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala index c21c6de32c0b..0c307b2b8576 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala @@ -243,6 +243,7 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { val b = AttributeReference("b", IntegerType)() checkMetadata(CreateStruct(Seq(a, b))) checkMetadata(CreateNamedStruct(Seq("a", a, "b", b))) + checkMetadata(CreateStructUnsafe(Seq(a, b))) checkMetadata(CreateNamedStructUnsafe(Seq("a", a, "b", b))) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 067b0bac6303..05e867bf5be9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -183,9 +183,6 @@ class Column(protected[sql] val expr: Expression) extends Logging { case a: AggregateExpression if a.aggregateFunction.isInstanceOf[TypedAggregateExpression] => UnresolvedAlias(a, Some(Column.generateAlias)) - // Wait until the struct is resolved. This will generate a nicer looking alias. - case struct: CreateNamedStructLike => UnresolvedAlias(struct) - case expr: Expression => Alias(expr, usePrettyExpression(expr).sql)() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala index 6141fab4aff0..f873f34a845e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala @@ -137,7 +137,7 @@ object ColumnStatStruct { private def numTrues(e: Expression): Expression = Sum(If(e, one, zero)) private def numFalses(e: Expression): Expression = Sum(If(Not(e), one, zero)) - private def getStruct(exprs: Seq[Expression]): CreateNamedStruct = { + private def getStruct(exprs: Seq[Expression]): CreateStruct = { CreateStruct(exprs.map { expr: Expression => expr.transformUp { case af: AggregateFunction => af.toAggregateExpression() @@ -168,7 +168,7 @@ object ColumnStatStruct { } } - def apply(attr: Attribute, relativeSD: Double): CreateNamedStruct = attr.dataType match { + def apply(attr: Attribute, relativeSD: Double): CreateStruct = attr.dataType match { // Use aggregate functions to compute statistics we need. case _: NumericType | TimestampType | DateType => getStruct(numericColumnStat(attr, relativeSD)) case StringType => getStruct(stringColumnStat(attr, relativeSD)) diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index d496af686d75..6741703d9d82 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -14,4 +14,4 @@ select 'foo' from myview where int_col == 0 group by 1; select 'foo', approx_count_distinct(int_col) from myview where int_col == 0 group by 1; -- group-by should not produce any rows (sort aggregate). -select 'foo', max(struct(int_col)) as agg_struct from myview where int_col == 0 group by 1; +select 'foo', max(struct(int_col)) from myview where int_col == 0 group by 1; diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index dede3a09ce75..9127bd4dd4c6 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -44,8 +44,8 @@ struct -- !query 5 -select 'foo', max(struct(int_col)) as agg_struct from myview where int_col == 0 group by 1 +select 'foo', max(struct(int_col)) from myview where int_col == 0 group by 1 -- !query 5 schema -struct> +struct> -- !query 5 output diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 90000445dffb..6eb571b91ffa 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -190,12 +190,6 @@ private[hive] class TestHiveSparkSession( new File(Thread.currentThread().getContextClassLoader.getResource(path).getFile) } - private def quoteHiveFile(path : String) = if (Utils.isWindows) { - getHiveFile(path).getPath.replace('\\', '/') - } else { - getHiveFile(path).getPath - } - def getWarehousePath(): String = { val tempConf = new SQLConf sc.conf.getAll.foreach { case (k, v) => tempConf.setConfString(k, v) } @@ -231,16 +225,16 @@ private[hive] class TestHiveSparkSession( val hiveQTestUtilTables: Seq[TestTable] = Seq( TestTable("src", "CREATE TABLE src (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' INTO TABLE src".cmd), + s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' INTO TABLE src".cmd), TestTable("src1", "CREATE TABLE src1 (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), + s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), TestTable("srcpart", () => { sql( "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { sql( - s"""LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' + s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') """.stripMargin) } @@ -250,7 +244,7 @@ private[hive] class TestHiveSparkSession( "CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { sql( - s"""LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' + s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') """.stripMargin) } @@ -275,7 +269,7 @@ private[hive] class TestHiveSparkSession( sql( s""" - |LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/complex.seq")}' + |LOAD DATA LOCAL INPATH '${getHiveFile("data/files/complex.seq")}' |INTO TABLE src_thrift """.stripMargin) }), @@ -314,7 +308,7 @@ private[hive] class TestHiveSparkSession( |) """.stripMargin.cmd, s""" - |LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/episodes.avro")}' + |LOAD DATA LOCAL INPATH '${getHiveFile("data/files/episodes.avro")}' |INTO TABLE episodes """.stripMargin.cmd ), @@ -385,7 +379,7 @@ private[hive] class TestHiveSparkSession( TestTable("src_json", s"""CREATE TABLE src_json (json STRING) STORED AS TEXTFILE """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/json.txt")}' INTO TABLE src_json".cmd) + s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/json.txt")}' INTO TABLE src_json".cmd) ) hiveQTestUtilTables.foreach(registerTestTable) diff --git a/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql index cdda29af50e3..de0116a4dcba 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql @@ -7,4 +7,4 @@ having b.key in (select a.key where a.value > 'val_9' and a.value = min(b.value)) order by b.key -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_5`) AS `gen_attr_1`, min(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (named_struct('gen_attr_0', `gen_attr_0`, 'gen_attr_4', `gen_attr_4`) IN (SELECT `gen_attr_6` AS `_c0`, `gen_attr_7` AS `_c1` FROM (SELECT `gen_attr_2` AS `gen_attr_6`, `gen_attr_3` AS `gen_attr_7` FROM (SELECT `gen_attr_2`, `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_3` > 'val_9')) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC NULLS FIRST) AS b +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_5`) AS `gen_attr_1`, min(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (struct(`gen_attr_0`, `gen_attr_4`) IN (SELECT `gen_attr_6` AS `_c0`, `gen_attr_7` AS `_c1` FROM (SELECT `gen_attr_2` AS `gen_attr_6`, `gen_attr_3` AS `gen_attr_7` FROM (SELECT `gen_attr_2`, `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_3` > 'val_9')) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC NULLS FIRST) AS b diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index 12d18dc87ceb..c7f10e569fa4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.catalyst import java.nio.charset.StandardCharsets import java.nio.file.{Files, NoSuchFileException, Paths} -import scala.io.Source import scala.util.control.NonFatal import org.apache.spark.sql.Column @@ -110,15 +109,12 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { Files.write(path, answerText.getBytes(StandardCharsets.UTF_8)) } else { val goldenFileName = s"sqlgen/$answerFile.sql" - val resourceStream = getClass.getClassLoader.getResourceAsStream(goldenFileName) - if (resourceStream == null) { + val resourceFile = getClass.getClassLoader.getResource(goldenFileName) + if (resourceFile == null) { throw new NoSuchFileException(goldenFileName) } - val answerText = try { - Source.fromInputStream(resourceStream).mkString - } finally { - resourceStream.close - } + val path = resourceFile.getPath + val answerText = new String(Files.readAllBytes(Paths.get(path)), StandardCharsets.UTF_8) val sqls = answerText.split(separator) assert(sqls.length == 2, "Golden sql files should have a separator.") val expectedSQL = sqls(1).trim() From 8ac09108fcf3fb62a812333a5b386b566a9d98ec Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Tue, 1 Nov 2016 10:46:36 -0700 Subject: [PATCH 025/132] [SPARK-17848][ML] Move LabelCol datatype cast into Predictor.fit ## What changes were proposed in this pull request? 1, move cast to `Predictor` 2, and then, remove unnecessary cast ## How was this patch tested? existing tests Author: Zheng RuiFeng Closes #15414 from zhengruifeng/move_cast. --- .../scala/org/apache/spark/ml/Predictor.scala | 12 ++- .../spark/ml/classification/Classifier.scala | 4 +- .../ml/classification/GBTClassifier.scala | 2 +- .../classification/LogisticRegression.scala | 2 +- .../spark/ml/classification/NaiveBayes.scala | 2 +- .../GeneralizedLinearRegression.scala | 2 +- .../ml/regression/LinearRegression.scala | 2 +- .../org/apache/spark/ml/PredictorSuite.scala | 82 +++++++++++++++++++ .../LogisticRegressionSuite.scala | 1 - 9 files changed, 98 insertions(+), 11 deletions(-) create mode 100644 mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala index e29d7f48a1d6..aa92edde7acd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -58,7 +58,8 @@ private[ml] trait PredictorParams extends Params /** * :: DeveloperApi :: - * Abstraction for prediction problems (regression and classification). + * Abstraction for prediction problems (regression and classification). It accepts all NumericType + * labels and will automatically cast it to DoubleType in [[fit()]]. * * @tparam FeaturesType Type of features. * E.g., [[org.apache.spark.mllib.linalg.VectorUDT]] for vector features. @@ -87,7 +88,12 @@ abstract class Predictor[ // This handles a few items such as schema validation. // Developers only need to implement train(). transformSchema(dataset.schema, logging = true) - copyValues(train(dataset).setParent(this)) + + // Cast LabelCol to DoubleType and keep the metadata. + val labelMeta = dataset.schema($(labelCol)).metadata + val casted = dataset.withColumn($(labelCol), col($(labelCol)).cast(DoubleType), labelMeta) + + copyValues(train(casted).setParent(this)) } override def copy(extra: ParamMap): Learner @@ -121,7 +127,7 @@ abstract class Predictor[ * and put it in an RDD with strong types. */ protected def extractLabeledPoints(dataset: Dataset[_]): RDD[LabeledPoint] = { - dataset.select(col($(labelCol)).cast(DoubleType), col($(featuresCol))).rdd.map { + dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { case Row(label: Double, features: Vector) => LabeledPoint(label, features) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index d1b21b16f234..a3da3067e1b5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -71,7 +71,7 @@ abstract class Classifier[ * and put it in an RDD with strong types. * * @param dataset DataFrame with columns for labels ([[org.apache.spark.sql.types.NumericType]]) - * and features ([[Vector]]). Labels are cast to [[DoubleType]]. + * and features ([[Vector]]). * @param numClasses Number of classes label can take. Labels must be integers in the range * [0, numClasses). * @throws SparkException if any label is not an integer >= 0 @@ -79,7 +79,7 @@ abstract class Classifier[ protected def extractLabeledPoints(dataset: Dataset[_], numClasses: Int): RDD[LabeledPoint] = { require(numClasses > 0, s"Classifier (in extractLabeledPoints) found numClasses =" + s" $numClasses, but requires numClasses > 0.") - dataset.select(col($(labelCol)).cast(DoubleType), col($(featuresCol))).rdd.map { + dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { case Row(label: Double, features: Vector) => require(label % 1 == 0 && label >= 0 && label < numClasses, s"Classifier was given" + s" dataset with invalid label $label. Labels must be integers in range" + diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 8bffe0cda032..f8f164e8c14b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -128,7 +128,7 @@ class GBTClassifier @Since("1.4.0") ( // We copy and modify this from Classifier.extractLabeledPoints since GBT only supports // 2 classes now. This lets us provide a more precise error message. val oldDataset: RDD[LabeledPoint] = - dataset.select(col($(labelCol)).cast(DoubleType), col($(featuresCol))).rdd.map { + dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { case Row(label: Double, features: Vector) => require(label == 0 || label == 1, s"GBTClassifier was given" + s" dataset with invalid label $label. Labels must be in {0,1}; note that" + diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 8fdaae04c42e..c4651054fd76 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -322,7 +322,7 @@ class LogisticRegression @Since("1.2.0") ( LogisticRegressionModel = { val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) val instances: RDD[Instance] = - dataset.select(col($(labelCol)).cast(DoubleType), w, col($(featuresCol))).rdd.map { + dataset.select(col($(labelCol)), w, col($(featuresCol))).rdd.map { case Row(label: Double, weight: Double, features: Vector) => Instance(label, weight, features) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 994ed993c99d..b03a07a6bc1e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -171,7 +171,7 @@ class NaiveBayes @Since("1.5.0") ( // Aggregates term frequencies per label. // TODO: Calling aggregateByKey and collect creates two stages, we can implement something // TODO: similar to reduceByKeyLocally to save one stage. - val aggregated = dataset.select(col($(labelCol)).cast(DoubleType), w, col($(featuresCol))).rdd + val aggregated = dataset.select(col($(labelCol)), w, col($(featuresCol))).rdd .map { row => (row.getDouble(0), (row.getDouble(1), row.getAs[Vector](2))) }.aggregateByKey[(Double, DenseVector)]((0.0, Vectors.zeros(numFeatures).toDense))( seqOp = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index 33cb25c8c7f6..8656ecf609ea 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -255,7 +255,7 @@ class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") override val val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) val instances: RDD[Instance] = - dataset.select(col($(labelCol)).cast(DoubleType), w, col($(featuresCol))).rdd.map { + dataset.select(col($(labelCol)), w, col($(featuresCol))).rdd.map { case Row(label: Double, weight: Double, features: Vector) => Instance(label, weight, features) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 519f3bdec82d..ae876b383973 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -190,7 +190,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) val instances: RDD[Instance] = dataset.select( - col($(labelCol)).cast(DoubleType), w, col($(featuresCol))).rdd.map { + col($(labelCol)), w, col($(featuresCol))).rdd.map { case Row(label: Double, weight: Double, features: Vector) => Instance(label, weight, features) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala new file mode 100644 index 000000000000..03e0c536a973 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala @@ -0,0 +1,82 @@ +/* + * 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.ml + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.linalg._ +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +class PredictorSuite extends SparkFunSuite with MLlibTestSparkContext { + + import PredictorSuite._ + + test("should support all NumericType labels and not support other types") { + val df = spark.createDataFrame(Seq( + (0, Vectors.dense(0, 2, 3)), + (1, Vectors.dense(0, 3, 9)), + (0, Vectors.dense(0, 2, 6)) + )).toDF("label", "features") + + val types = + Seq(ShortType, LongType, IntegerType, FloatType, ByteType, DoubleType, DecimalType(10, 0)) + + val predictor = new MockPredictor() + + types.foreach { t => + predictor.fit(df.select(col("label").cast(t), col("features"))) + } + + intercept[IllegalArgumentException] { + predictor.fit(df.select(col("label").cast(StringType), col("features"))) + } + } +} + +object PredictorSuite { + + class MockPredictor(override val uid: String) + extends Predictor[Vector, MockPredictor, MockPredictionModel] { + + def this() = this(Identifiable.randomUID("mockpredictor")) + + override def train(dataset: Dataset[_]): MockPredictionModel = { + require(dataset.schema("label").dataType == DoubleType) + new MockPredictionModel(uid) + } + + override def copy(extra: ParamMap): MockPredictor = + throw new NotImplementedError() + } + + class MockPredictionModel(override val uid: String) + extends PredictionModel[Vector, MockPredictionModel] { + + def this() = this(Identifiable.randomUID("mockpredictormodel")) + + override def predict(features: Vector): Double = + throw new NotImplementedError() + + override def copy(extra: ParamMap): MockPredictionModel = + throw new NotImplementedError() + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index bc631dc6d314..8771fd2e9d2b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -1807,7 +1807,6 @@ class LogisticRegressionSuite .objectiveHistory .sliding(2) .forall(x => x(0) >= x(1))) - } test("binary logistic regression with weighted data") { From 8cdf143f4b1ca5c6bc0256808e6f42d9ef299cbd Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Tue, 1 Nov 2016 11:17:35 -0700 Subject: [PATCH 026/132] [SPARK-18103][FOLLOW-UP][SQL][MINOR] Rename `MetadataLogFileCatalog` to `MetadataLogFileIndex` ## What changes were proposed in this pull request? This is a follow-up to https://github.com/apache/spark/pull/15634. ## How was this patch tested? N/A Author: Liwei Lin Closes #15712 from lw-lin/18103. --- .../{MetadataLogFileCatalog.scala => MetadataLogFileIndex.scala} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/{MetadataLogFileCatalog.scala => MetadataLogFileIndex.scala} (100%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala similarity index 100% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileCatalog.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala From 8a538c97b556f80f67c80519af0ce879557050d5 Mon Sep 17 00:00:00 2001 From: Ergin Seyfe Date: Tue, 1 Nov 2016 11:18:42 -0700 Subject: [PATCH 027/132] [SPARK-18189][SQL] Fix serialization issue in KeyValueGroupedDataset ## What changes were proposed in this pull request? Likewise [DataSet.scala](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala#L156) KeyValueGroupedDataset should mark the queryExecution as transient. As mentioned in the Jira ticket, without transient we saw serialization issues like ``` Caused by: java.io.NotSerializableException: org.apache.spark.sql.execution.QueryExecution Serialization stack: - object not serializable (class: org.apache.spark.sql.execution.QueryExecution, value: == ``` ## How was this patch tested? Run the query which is specified in the Jira ticket before and after: ``` val a = spark.createDataFrame(sc.parallelize(Seq((1,2),(3,4)))).as[(Int,Int)] val grouped = a.groupByKey( {x:(Int,Int)=>x._1} ) val mappedGroups = grouped.mapGroups((k,x)=> {(k,1)} ) val yyy = sc.broadcast(1) val last = mappedGroups.rdd.map(xx=> { val simpley = yyy.value 1 } ) ``` Author: Ergin Seyfe Closes #15706 from seyfe/keyvaluegrouped_serialization. --- .../scala/org/apache/spark/repl/ReplSuite.scala | 17 +++++++++++++++++ .../spark/sql/KeyValueGroupedDataset.scala | 2 +- 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 9262e938c2a6..96d2dfc2658b 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -473,4 +473,21 @@ class ReplSuite extends SparkFunSuite { assertDoesNotContain("AssertionError", output) assertDoesNotContain("Exception", output) } + + test("SPARK-18189: Fix serialization issue in KeyValueGroupedDataset") { + val resultValue = 12345 + val output = runInterpreter("local", + s""" + |val keyValueGrouped = Seq((1, 2), (3, 4)).toDS().groupByKey(_._1) + |val mapGroups = keyValueGrouped.mapGroups((k, v) => (k, 1)) + |val broadcasted = sc.broadcast($resultValue) + | + |// Using broadcast triggers serialization issue in KeyValueGroupedDataset + |val dataset = mapGroups.map(_ => broadcasted.value) + |dataset.collect() + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + assertContains(s": Array[Int] = Array($resultValue, $resultValue)", output) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 4cb0313aa903..31ce8eb25e80 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -40,7 +40,7 @@ import org.apache.spark.sql.expressions.ReduceAggregator class KeyValueGroupedDataset[K, V] private[sql]( kEncoder: Encoder[K], vEncoder: Encoder[V], - val queryExecution: QueryExecution, + @transient val queryExecution: QueryExecution, private val dataAttributes: Seq[Attribute], private val groupingAttributes: Seq[Attribute]) extends Serializable { From d0272b436512b71f04313e109d3d21a6e9deefca Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Tue, 1 Nov 2016 11:25:11 -0700 Subject: [PATCH 028/132] [SPARK-18148][SQL] Misleading Error Message for Aggregation Without Window/GroupBy ## What changes were proposed in this pull request? Aggregation Without Window/GroupBy expressions will fail in `checkAnalysis`, the error message is a bit misleading, we should generate a more specific error message for this case. For example, ``` spark.read.load("/some-data") .withColumn("date_dt", to_date($"date")) .withColumn("year", year($"date_dt")) .withColumn("week", weekofyear($"date_dt")) .withColumn("user_count", count($"userId")) .withColumn("daily_max_in_week", max($"user_count").over(weeklyWindow)) ) ``` creates the following output: ``` org.apache.spark.sql.AnalysisException: expression '`randomColumn`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; ``` In the error message above, `randomColumn` doesn't appear in the query(acturally it's added by function `withColumn`), so the message is not enough for the user to address the problem. ## How was this patch tested? Manually test Before: ``` scala> spark.sql("select col, count(col) from tbl") org.apache.spark.sql.AnalysisException: expression 'tbl.`col`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;; ``` After: ``` scala> spark.sql("select col, count(col) from tbl") org.apache.spark.sql.AnalysisException: grouping expressions sequence is empty, and 'tbl.`col`' is not an aggregate function. Wrap '(count(col#231L) AS count(col)#239L)' in windowing function(s) or wrap 'tbl.`col`' in first() (or first_value) if you don't care which value you get.;; ``` Also add new test sqls in `group-by.sql`. Author: jiangxingbo Closes #15672 from jiangxb1987/groupBy-empty. --- .../sql/catalyst/analysis/CheckAnalysis.scala | 12 ++ .../resources/sql-tests/inputs/group-by.sql | 41 +++++-- .../sql-tests/results/group-by.sql.out | 116 +++++++++++++++--- .../org/apache/spark/sql/SQLQuerySuite.scala | 35 ------ 4 files changed, 140 insertions(+), 64 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 9a7c2a944b58..3455a567b778 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -214,6 +214,18 @@ trait CheckAnalysis extends PredicateHelper { s"appear in the arguments of an aggregate function.") } } + case e: Attribute if groupingExprs.isEmpty => + // Collect all [[AggregateExpressions]]s. + val aggExprs = aggregateExprs.filter(_.collect { + case a: AggregateExpression => a + }.nonEmpty) + failAnalysis( + s"grouping expressions sequence is empty, " + + s"and '${e.sql}' is not an aggregate function. " + + s"Wrap '${aggExprs.map(_.sql).mkString("(", ", ", ")")}' in windowing " + + s"function(s) or wrap '${e.sql}' in first() (or first_value) " + + s"if you don't care which value you get." + ) case e: Attribute if !groupingExprs.exists(_.semanticEquals(e)) => failAnalysis( s"expression '${e.sql}' is neither present in the group by, " + diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index 6741703d9d82..d950ec83d98c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -1,17 +1,34 @@ --- Temporary data. -create temporary view myview as values 128, 256 as v(int_col); +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b); --- group by should produce all input rows, -select int_col, count(*) from myview group by int_col; +-- Aggregate with empty GroupBy expressions. +SELECT a, COUNT(b) FROM testData; +SELECT COUNT(a), COUNT(b) FROM testData; --- group by should produce a single row. -select 'foo', count(*) from myview group by 1; +-- Aggregate with non-empty GroupBy expressions. +SELECT a, COUNT(b) FROM testData GROUP BY a; +SELECT a, COUNT(b) FROM testData GROUP BY b; +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a; --- group-by should not produce any rows (whole stage code generation). -select 'foo' from myview where int_col == 0 group by 1; +-- Aggregate grouped by literals. +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1; --- group-by should not produce any rows (hash aggregate). -select 'foo', approx_count_distinct(int_col) from myview where int_col == 0 group by 1; +-- Aggregate grouped by literals (whole stage code generation). +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1; --- group-by should not produce any rows (sort aggregate). -select 'foo', max(struct(int_col)) from myview where int_col == 0 group by 1; +-- Aggregate grouped by literals (hash aggregate). +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate grouped by literals (sort aggregate). +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1; + +-- Aggregate with complex GroupBy expressions. +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b; +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1; +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1; + +-- Aggregate with nulls. +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData; diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 9127bd4dd4c6..a91f04e098b1 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -1,9 +1,11 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 6 +-- Number of queries: 14 -- !query 0 -create temporary view myview as values 128, 256 as v(int_col) +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2), (null, 1), (3, null), (null, null) +AS testData(a, b) -- !query 0 schema struct<> -- !query 0 output @@ -11,41 +13,121 @@ struct<> -- !query 1 -select int_col, count(*) from myview group by int_col +SELECT a, COUNT(b) FROM testData -- !query 1 schema -struct +struct<> -- !query 1 output -128 1 -256 1 +org.apache.spark.sql.AnalysisException +grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(count(testdata.`b`) AS `count(b)`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.; -- !query 2 -select 'foo', count(*) from myview group by 1 +SELECT COUNT(a), COUNT(b) FROM testData -- !query 2 schema -struct +struct -- !query 2 output -foo 2 +7 7 -- !query 3 -select 'foo' from myview where int_col == 0 group by 1 +SELECT a, COUNT(b) FROM testData GROUP BY a -- !query 3 schema -struct +struct -- !query 3 output - +1 2 +2 2 +3 2 +NULL 1 -- !query 4 -select 'foo', approx_count_distinct(int_col) from myview where int_col == 0 group by 1 +SELECT a, COUNT(b) FROM testData GROUP BY b -- !query 4 schema -struct +struct<> -- !query 4 output - +org.apache.spark.sql.AnalysisException +expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; -- !query 5 -select 'foo', max(struct(int_col)) from myview where int_col == 0 group by 1 +SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a -- !query 5 schema -struct> +struct -- !query 5 output +0 1 +2 2 +2 2 +3 2 + + +-- !query 6 +SELECT 'foo', COUNT(a) FROM testData GROUP BY 1 +-- !query 6 schema +struct +-- !query 6 output +foo 7 + + +-- !query 7 +SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1 +-- !query 7 schema +struct +-- !query 7 output + + +-- !query 8 +SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1 +-- !query 8 schema +struct +-- !query 8 output + + + +-- !query 9 +SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1 +-- !query 9 schema +struct> +-- !query 9 output + + + +-- !query 10 +SELECT a + b, COUNT(b) FROM testData GROUP BY a + b +-- !query 10 schema +struct<(a + b):int,count(b):bigint> +-- !query 10 output +2 1 +3 2 +4 2 +5 1 +NULL 1 + + +-- !query 11 +SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1 +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.AnalysisException +expression 'testdata.`a`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.; + + +-- !query 12 +SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1 +-- !query 12 schema +struct<((a + 1) + 1):int,count(b):bigint> +-- !query 12 output +3 2 +4 2 +5 2 +NULL 1 + + +-- !query 13 +SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) +FROM testData +-- !query 13 schema +struct +-- !query 13 output +-0.2723801058145729 -1.5069204152249134 1 3 2.142857142857143 0.8095238095238094 0.8997354108424372 15 7 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 1a43d0b2205c..9a3d93cf17b7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -463,20 +463,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { ) } - test("agg") { - checkAnswer( - sql("SELECT a, SUM(b) FROM testData2 GROUP BY a"), - Seq(Row(1, 3), Row(2, 3), Row(3, 3))) - } - - test("aggregates with nulls") { - checkAnswer( - sql("SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a)," + - "AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a) FROM nullInts"), - Row(0, -1.5, 1, 3, 2, 1.0, 1, 6, 3) - ) - } - test("select *") { checkAnswer( sql("SELECT * FROM testData"), @@ -1178,27 +1164,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { Row(1)) } - test("throw errors for non-aggregate attributes with aggregation") { - def checkAggregation(query: String, isInvalidQuery: Boolean = true) { - if (isInvalidQuery) { - val e = intercept[AnalysisException](sql(query).queryExecution.analyzed) - assert(e.getMessage contains "group by") - } else { - // Should not throw - sql(query).queryExecution.analyzed - } - } - - checkAggregation("SELECT key, COUNT(*) FROM testData") - checkAggregation("SELECT COUNT(key), COUNT(*) FROM testData", isInvalidQuery = false) - - checkAggregation("SELECT value, COUNT(*) FROM testData GROUP BY key") - checkAggregation("SELECT COUNT(value), SUM(key) FROM testData GROUP BY key", false) - - checkAggregation("SELECT key + 2, COUNT(*) FROM testData GROUP BY key + 1") - checkAggregation("SELECT key + 1 + 1, COUNT(*) FROM testData GROUP BY key + 1", false) - } - testQuietly( "SPARK-16748: SparkExceptions during planning should not wrapped in TreeNodeException") { intercept[SparkException] { From cfac17ee1cec414663b957228e469869eb7673c1 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 1 Nov 2016 12:35:34 -0700 Subject: [PATCH 029/132] [SPARK-18167] Disable flaky SQLQuerySuite test We now know it's a persistent environmental issue that is causing this test to sometimes fail. One hypothesis is that some configuration is leaked from another suite, and depending on suite ordering this can cause this test to fail. I am planning on mining the jenkins logs to try to narrow down which suite could be causing this. For now, disable the test. Author: Eric Liang Closes #15720 from ericl/disable-flaky-test. --- .../org/apache/spark/sql/hive/execution/SQLQuerySuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 8b916932ff54..b9353b5b5d2a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -1565,7 +1565,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { ).map(i => Row(i._1, i._2, i._3, i._4))) } - test("SPARK-10562: partition by column with mixed case name") { + ignore("SPARK-10562: partition by column with mixed case name") { def runOnce() { withTable("tbl10562") { val df = Seq(2012 -> "a").toDF("Year", "val") From 01dd0083011741c2bbe5ae1d2a25f2c9a1302b76 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Tue, 1 Nov 2016 12:46:41 -0700 Subject: [PATCH 030/132] [SPARK-17764][SQL] Add `to_json` supporting to convert nested struct column to JSON string ## What changes were proposed in this pull request? This PR proposes to add `to_json` function in contrast with `from_json` in Scala, Java and Python. It'd be useful if we can convert a same column from/to json. Also, some datasources do not support nested types. If we are forced to save a dataframe into those data sources, we might be able to work around by this function. The usage is as below: ``` scala val df = Seq(Tuple1(Tuple1(1))).toDF("a") df.select(to_json($"a").as("json")).show() ``` ``` bash +--------+ | json| +--------+ |{"_1":1}| +--------+ ``` ## How was this patch tested? Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite`. Author: hyukjinkwon Closes #15354 from HyukjinKwon/SPARK-17764. --- python/pyspark/sql/functions.py | 23 +++++++++ python/pyspark/sql/readwriter.py | 2 +- python/pyspark/sql/streaming.py | 2 +- .../expressions/jsonExpressions.scala | 48 ++++++++++++++++++- .../sql/catalyst}/json/JacksonGenerator.scala | 5 +- .../sql/catalyst/json/JacksonUtils.scala | 26 ++++++++++ .../expressions/JsonExpressionsSuite.scala | 9 ++++ .../scala/org/apache/spark/sql/Dataset.scala | 2 +- .../datasources/json/JsonFileFormat.scala | 2 +- .../org/apache/spark/sql/functions.scala | 44 ++++++++++++++++- .../apache/spark/sql/JsonFunctionsSuite.scala | 30 +++++++++--- 11 files changed, 177 insertions(+), 16 deletions(-) rename sql/{core/src/main/scala/org/apache/spark/sql/execution/datasources => catalyst/src/main/scala/org/apache/spark/sql/catalyst}/json/JacksonGenerator.scala (98%) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 7fa3fd2de7dd..45e3c22bfc6a 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1744,6 +1744,29 @@ def from_json(col, schema, options={}): return Column(jc) +@ignore_unicode_prefix +@since(2.1) +def to_json(col, options={}): + """ + Converts a column containing a [[StructType]] into a JSON string. Throws an exception, + in the case of an unsupported type. + + :param col: name of column containing the struct + :param options: options to control converting. accepts the same options as the json datasource + + >>> from pyspark.sql import Row + >>> from pyspark.sql.types import * + >>> data = [(1, Row(name='Alice', age=2))] + >>> df = spark.createDataFrame(data, ("key", "value")) + >>> df.select(to_json(df.value).alias("json")).collect() + [Row(json=u'{"age":2,"name":"Alice"}')] + """ + + sc = SparkContext._active_spark_context + jc = sc._jvm.functions.to_json(_to_java_column(col), options) + return Column(jc) + + @since(1.5) def size(col): """ diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index bc786ef95ed0..b0c51b1e9992 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -161,7 +161,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None): """ Loads a JSON file (`JSON Lines text format or newline-delimited JSON - <[http://jsonlines.org/>`_) or an RDD of Strings storing JSON objects (one object per + `_) or an RDD of Strings storing JSON objects (one object per record) and returns the result as a :class`DataFrame`. If the ``schema`` parameter is not specified, this function goes diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 559647bbabf6..1c94413e3c45 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -641,7 +641,7 @@ def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None, timestampFormat=None): """ Loads a JSON file stream (`JSON Lines text format or newline-delimited JSON - <[http://jsonlines.org/>`_) and returns a :class`DataFrame`. + `_) and returns a :class`DataFrame`. If the ``schema`` parameter is not specified, this function goes through the input once to determine the input schema. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index 65dbd6a4e3f1..244a5a34f359 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -17,16 +17,17 @@ package org.apache.spark.sql.catalyst.expressions -import java.io.{ByteArrayOutputStream, StringWriter} +import java.io.{ByteArrayOutputStream, CharArrayWriter, StringWriter} import scala.util.parsing.combinator.RegexParsers import com.fasterxml.jackson.core._ +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.json.{JacksonParser, JSONOptions, SparkSQLJsonProcessingException} +import org.apache.spark.sql.catalyst.json._ import org.apache.spark.sql.catalyst.util.ParseModes import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -494,3 +495,46 @@ case class JsonToStruct(schema: StructType, options: Map[String, String], child: override def inputTypes: Seq[AbstractDataType] = StringType :: Nil } + +/** + * Converts a [[StructType]] to a json output string. + */ +case class StructToJson(options: Map[String, String], child: Expression) + extends Expression with CodegenFallback with ExpectsInputTypes { + override def nullable: Boolean = true + + @transient + lazy val writer = new CharArrayWriter() + + @transient + lazy val gen = + new JacksonGenerator(child.dataType.asInstanceOf[StructType], writer) + + override def dataType: DataType = StringType + override def children: Seq[Expression] = child :: Nil + + override def checkInputDataTypes(): TypeCheckResult = { + if (StructType.acceptsType(child.dataType)) { + try { + JacksonUtils.verifySchema(child.dataType.asInstanceOf[StructType]) + TypeCheckResult.TypeCheckSuccess + } catch { + case e: UnsupportedOperationException => + TypeCheckResult.TypeCheckFailure(e.getMessage) + } + } else { + TypeCheckResult.TypeCheckFailure( + s"$prettyName requires that the expression is a struct expression.") + } + } + + override def eval(input: InternalRow): Any = { + gen.write(child.eval(input).asInstanceOf[InternalRow]) + gen.flush() + val json = writer.toString + writer.reset() + UTF8String.fromString(json) + } + + override def inputTypes: Seq[AbstractDataType] = StructType :: Nil +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala similarity index 98% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala index 5b55b701862b..4b548e0e7f97 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JacksonGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonGenerator.scala @@ -15,15 +15,14 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.datasources.json +package org.apache.spark.sql.catalyst.json import java.io.Writer import com.fasterxml.jackson.core._ -import org.apache.spark.sql.catalyst.expressions.SpecializedGetters import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.json.JSONOptions +import org.apache.spark.sql.catalyst.expressions.SpecializedGetters import org.apache.spark.sql.catalyst.util.{ArrayData, DateTimeUtils, MapData} import org.apache.spark.sql.types._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonUtils.scala index c4d9abb2c07e..3b23c6cd2816 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonUtils.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.json import com.fasterxml.jackson.core.{JsonParser, JsonToken} +import org.apache.spark.sql.types._ + object JacksonUtils { /** * Advance the parser until a null or a specific token is found @@ -29,4 +31,28 @@ object JacksonUtils { case x => x != stopOn } } + + /** + * Verify if the schema is supported in JSON parsing. + */ + def verifySchema(schema: StructType): Unit = { + def verifyType(name: String, dataType: DataType): Unit = dataType match { + case NullType | BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | + DoubleType | StringType | TimestampType | DateType | BinaryType | _: DecimalType => + + case st: StructType => st.foreach(field => verifyType(field.name, field.dataType)) + + case at: ArrayType => verifyType(name, at.elementType) + + case mt: MapType => verifyType(name, mt.keyType) + + case udt: UserDefinedType[_] => verifyType(name, udt.sqlType) + + case _ => + throw new UnsupportedOperationException( + s"Unable to convert column $name of type ${dataType.simpleString} to JSON.") + } + + schema.foreach(field => verifyType(field.name, field.dataType)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala index 84623934d95d..f9db649bc240 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/JsonExpressionsSuite.scala @@ -343,4 +343,13 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { null ) } + + test("to_json") { + val schema = StructType(StructField("a", IntegerType) :: Nil) + val struct = Literal.create(create_row(1), schema) + checkEvaluation( + StructToJson(Map.empty, struct), + """{"a":1}""" + ) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 6e0a2471e0fb..eb2b20afc37c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.json.JacksonGenerator import org.apache.spark.sql.catalyst.optimizer.CombineUnions import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -45,7 +46,6 @@ import org.apache.spark.sql.catalyst.util.usePrettyExpression import org.apache.spark.sql.execution.{FileRelation, LogicalRDD, QueryExecution, SQLExecution} import org.apache.spark.sql.execution.command.{CreateViewCommand, ExplainCommand, GlobalTempView, LocalTempView} import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.execution.datasources.json.JacksonGenerator import org.apache.spark.sql.execution.python.EvaluatePython import org.apache.spark.sql.streaming.DataStreamWriter import org.apache.spark.sql.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index 5a409c04c929..0e38aefecb67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -32,7 +32,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.json.{JacksonParser, JSONOptions} +import org.apache.spark.sql.catalyst.json.{JacksonGenerator, JacksonParser, JSONOptions} import org.apache.spark.sql.catalyst.util.CompressionCodecs import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.text.TextOutputWriter diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 5f1efd22d820..944a476114fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -2883,10 +2883,10 @@ object functions { * (Scala-specific) Parses a column containing a JSON string into a [[StructType]] with the * specified schema. Returns `null`, in the case of an unparseable string. * + * @param e a string column containing JSON data. * @param schema the schema to use when parsing the json string * @param options options to control how the json is parsed. accepts the same options and the * json data source. - * @param e a string column containing JSON data. * * @group collection_funcs * @since 2.1.0 @@ -2936,6 +2936,48 @@ object functions { def from_json(e: Column, schema: String, options: java.util.Map[String, String]): Column = from_json(e, DataType.fromJson(schema).asInstanceOf[StructType], options) + + /** + * (Scala-specific) Converts a column containing a [[StructType]] into a JSON string with the + * specified schema. Throws an exception, in the case of an unsupported type. + * + * @param e a struct column. + * @param options options to control how the struct column is converted into a json string. + * accepts the same options and the json data source. + * + * @group collection_funcs + * @since 2.1.0 + */ + def to_json(e: Column, options: Map[String, String]): Column = withExpr { + StructToJson(options, e.expr) + } + + /** + * (Java-specific) Converts a column containing a [[StructType]] into a JSON string with the + * specified schema. Throws an exception, in the case of an unsupported type. + * + * @param e a struct column. + * @param options options to control how the struct column is converted into a json string. + * accepts the same options and the json data source. + * + * @group collection_funcs + * @since 2.1.0 + */ + def to_json(e: Column, options: java.util.Map[String, String]): Column = + to_json(e, options.asScala.toMap) + + /** + * Converts a column containing a [[StructType]] into a JSON string with the + * specified schema. Throws an exception, in the case of an unsupported type. + * + * @param e a struct column. + * + * @group collection_funcs + * @since 2.1.0 + */ + def to_json(e: Column): Column = + to_json(e, Map.empty[String, String]) + /** * Returns length of array or map. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index 518d6e92b2ff..59ae889cf3b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql -import org.apache.spark.sql.functions.from_json +import org.apache.spark.sql.functions.{from_json, struct, to_json} import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.types.{CalendarIntervalType, IntegerType, StructType} class JsonFunctionsSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -31,7 +31,6 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { Row("alice", "5")) } - val tuples: Seq[(String, String)] = ("1", """{"f1": "value1", "f2": "value2", "f3": 3, "f5": 5.23}""") :: ("2", """{"f1": "value12", "f3": "value3", "f2": 2, "f4": 4.01}""") :: @@ -97,7 +96,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { checkAnswer(expr, expected) } - test("json_parser") { + test("from_json") { val df = Seq("""{"a": 1}""").toDS() val schema = new StructType().add("a", IntegerType) @@ -106,7 +105,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { Row(Row(1)) :: Nil) } - test("json_parser missing columns") { + test("from_json missing columns") { val df = Seq("""{"a": 1}""").toDS() val schema = new StructType().add("b", IntegerType) @@ -115,7 +114,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { Row(Row(null)) :: Nil) } - test("json_parser invalid json") { + test("from_json invalid json") { val df = Seq("""{"a" 1}""").toDS() val schema = new StructType().add("a", IntegerType) @@ -123,4 +122,23 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { df.select(from_json($"value", schema)), Row(null) :: Nil) } + + test("to_json") { + val df = Seq(Tuple1(Tuple1(1))).toDF("a") + + checkAnswer( + df.select(to_json($"a")), + Row("""{"_1":1}""") :: Nil) + } + + test("to_json unsupported type") { + val df = Seq(Tuple1(Tuple1("interval -3 month 7 hours"))).toDF("a") + .select(struct($"a._1".cast(CalendarIntervalType).as("a")).as("c")) + val e = intercept[AnalysisException]{ + // Unsupported type throws an exception + df.select(to_json($"c")).collect() + } + assert(e.getMessage.contains( + "Unable to convert column a of type calendarinterval to JSON.")) + } } From 6e6298154aba63831a292117797798131a646869 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 1 Nov 2016 16:23:47 -0700 Subject: [PATCH 031/132] [SPARK-17350][SQL] Disable default use of KryoSerializer in Thrift Server In SPARK-4761 / #3621 (December 2014) we enabled Kryo serialization by default in the Spark Thrift Server. However, I don't think that the original rationale for doing this still holds now that most Spark SQL serialization is now performed via encoders and our UnsafeRow format. In addition, the use of Kryo as the default serializer can introduce performance problems because the creation of new KryoSerializer instances is expensive and we haven't performed instance-reuse optimizations in several code paths (including DirectTaskResult deserialization). Given all of this, I propose to revert back to using JavaSerializer as the default serializer in the Thrift Server. /cc liancheng Author: Josh Rosen Closes #14906 from JoshRosen/disable-kryo-in-thriftserver. --- docs/configuration.md | 5 ++--- .../spark/sql/hive/thriftserver/SparkSQLEnv.scala | 10 ---------- 2 files changed, 2 insertions(+), 13 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 780fc94908d3..0017219e0726 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -767,7 +767,7 @@ Apart from these, the following properties are also available, and may be useful spark.kryo.referenceTracking - true (false when using Spark SQL Thrift Server) + true Whether to track references to the same object when serializing data with Kryo, which is necessary if your object graphs have loops and useful for efficiency if they contain multiple @@ -838,8 +838,7 @@ Apart from these, the following properties are also available, and may be useful spark.serializer - org.apache.spark.serializer.
JavaSerializer (org.apache.spark.serializer.
- KryoSerializer when using Spark SQL Thrift Server) + org.apache.spark.serializer.
JavaSerializer Class to use for serializing objects that will be sent over the network or need to be cached diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 638911599aad..78a309497ab5 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.hive.thriftserver import java.io.PrintStream -import scala.collection.JavaConverters._ - import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.internal.Logging import org.apache.spark.sql.{SparkSession, SQLContext} @@ -37,8 +35,6 @@ private[hive] object SparkSQLEnv extends Logging { def init() { if (sqlContext == null) { val sparkConf = new SparkConf(loadDefaults = true) - val maybeSerializer = sparkConf.getOption("spark.serializer") - val maybeKryoReferenceTracking = sparkConf.getOption("spark.kryo.referenceTracking") // 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 @@ -47,12 +43,6 @@ private[hive] object SparkSQLEnv extends Logging { sparkConf .setAppName(maybeAppName.getOrElse(s"SparkSQL::${Utils.localHostName()}")) - .set( - "spark.serializer", - maybeSerializer.getOrElse("org.apache.spark.serializer.KryoSerializer")) - .set( - "spark.kryo.referenceTracking", - maybeKryoReferenceTracking.getOrElse("false")) val sparkSession = SparkSession.builder.config(sparkConf).enableHiveSupport().getOrCreate() sparkContext = sparkSession.sparkContext From b929537b6eb0f8f34497c3dbceea8045bf5dffdb Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 1 Nov 2016 16:49:41 -0700 Subject: [PATCH 032/132] [SPARK-18182] Expose ReplayListenerBus.read() overload which takes string iterator The `ReplayListenerBus.read()` method is used when implementing a custom `ApplicationHistoryProvider`. The current interface only exposes a `read()` method which takes an `InputStream` and performs stream-to-lines conversion itself, but it would also be useful to expose an overloaded method which accepts an iterator of strings, thereby enabling events to be provided from non-`InputStream` sources. Author: Josh Rosen Closes #15698 from JoshRosen/replay-listener-bus-interface. --- .../spark/scheduler/ReplayListenerBus.scala | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 2424586431aa..0bd5a6bc59a9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -53,13 +53,24 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { sourceName: String, maybeTruncated: Boolean = false, eventsFilter: ReplayEventsFilter = SELECT_ALL_FILTER): Unit = { + val lines = Source.fromInputStream(logData).getLines() + replay(lines, sourceName, maybeTruncated, eventsFilter) + } + /** + * Overloaded variant of [[replay()]] which accepts an iterator of lines instead of an + * [[InputStream]]. Exposed for use by custom ApplicationHistoryProvider implementations. + */ + def replay( + lines: Iterator[String], + sourceName: String, + maybeTruncated: Boolean, + eventsFilter: ReplayEventsFilter): Unit = { var currentLine: String = null var lineNumber: Int = 0 try { - val lineEntries = Source.fromInputStream(logData) - .getLines() + val lineEntries = lines .zipWithIndex .filter { case (line, _) => eventsFilter(line) } From 91c33a0ca5c8287f710076ed7681e5aa13ca068f Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 1 Nov 2016 17:00:00 -0700 Subject: [PATCH 033/132] [SPARK-18088][ML] Various ChiSqSelector cleanups ## What changes were proposed in this pull request? - Renamed kbest to numTopFeatures - Renamed alpha to fpr - Added missing Since annotations - Doc cleanups ## How was this patch tested? Added new standardized unit tests for spark.ml. Improved existing unit test coverage a bit. Author: Joseph K. Bradley Closes #15647 from jkbradley/chisqselector-follow-ups. --- docs/ml-features.md | 12 +- docs/mllib-feature-extraction.md | 15 +- .../spark/ml/feature/ChiSqSelector.scala | 59 ++++---- .../mllib/api/python/PythonMLLibAPI.scala | 4 +- .../spark/mllib/feature/ChiSqSelector.scala | 45 +++--- .../spark/ml/feature/ChiSqSelectorSuite.scala | 135 ++++++++++-------- .../mllib/feature/ChiSqSelectorSuite.scala | 17 +-- python/pyspark/ml/feature.py | 37 ++--- python/pyspark/mllib/feature.py | 58 ++++---- 9 files changed, 197 insertions(+), 185 deletions(-) diff --git a/docs/ml-features.md b/docs/ml-features.md index 64c6a160239c..352887d3ba6e 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -1338,14 +1338,14 @@ for more details on the API. `ChiSqSelector` stands for Chi-Squared feature selection. It operates on labeled data with categorical features. ChiSqSelector uses the [Chi-Squared test of independence](https://en.wikipedia.org/wiki/Chi-squared_test) to decide which -features to choose. It supports three selection methods: `KBest`, `Percentile` and `FPR`: +features to choose. It supports three selection methods: `numTopFeatures`, `percentile`, `fpr`: -* `KBest` chooses the `k` top features according to a chi-squared test. This is akin to yielding the features with the most predictive power. -* `Percentile` is similar to `KBest` but chooses a fraction of all features instead of a fixed number. -* `FPR` chooses all features whose false positive rate meets some threshold. +* `numTopFeatures` chooses a fixed number of top features according to a chi-squared test. This is akin to yielding the features with the most predictive power. +* `percentile` is similar to `numTopFeatures` but chooses a fraction of all features instead of a fixed number. +* `fpr` chooses all features whose p-value is below a threshold, thus controlling the false positive rate of selection. -By default, the selection method is `KBest`, the default number of top features is 50. User can use -`setNumTopFeatures`, `setPercentile` and `setAlpha` to set different selection methods. +By default, the selection method is `numTopFeatures`, with the default number of top features set to 50. +The user can choose a selection method using `setSelectorType`. **Examples** diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 87e1e027e945..42568c312e70 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -227,22 +227,19 @@ both speed and statistical learning behavior. [`ChiSqSelector`](api/scala/index.html#org.apache.spark.mllib.feature.ChiSqSelector) implements Chi-Squared feature selection. It operates on labeled data with categorical features. ChiSqSelector uses the [Chi-Squared test of independence](https://en.wikipedia.org/wiki/Chi-squared_test) to decide which -features to choose. It supports three selection methods: `KBest`, `Percentile` and `FPR`: +features to choose. It supports three selection methods: `numTopFeatures`, `percentile`, `fpr`: -* `KBest` chooses the `k` top features according to a chi-squared test. This is akin to yielding the features with the most predictive power. -* `Percentile` is similar to `KBest` but chooses a fraction of all features instead of a fixed number. -* `FPR` chooses all features whose false positive rate meets some threshold. +* `numTopFeatures` chooses a fixed number of top features according to a chi-squared test. This is akin to yielding the features with the most predictive power. +* `percentile` is similar to `numTopFeatures` but chooses a fraction of all features instead of a fixed number. +* `fpr` chooses all features whose p-value is below a threshold, thus controlling the false positive rate of selection. -By default, the selection method is `KBest`, the default number of top features is 50. User can use -`setNumTopFeatures`, `setPercentile` and `setAlpha` to set different selection methods. +By default, the selection method is `numTopFeatures`, with the default number of top features set to 50. +The user can choose a selection method using `setSelectorType`. The number of features to select can be tuned using a held-out validation set. ### Model Fitting -`ChiSqSelector` takes a `numTopFeatures` parameter specifying the number of top features that -the selector will select. - The [`fit`](api/scala/index.html#org.apache.spark.mllib.feature.ChiSqSelector) method takes an input of `RDD[LabeledPoint]` with categorical features, learns the summary statistics, and then returns a `ChiSqSelectorModel` which can transform an input dataset into the reduced feature space. diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index d0385e220e1e..653fa41124f8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -42,69 +42,80 @@ private[feature] trait ChiSqSelectorParams extends Params with HasFeaturesCol with HasOutputCol with HasLabelCol { /** - * Number of features that selector will select (ordered by statistic value descending). If the + * Number of features that selector will select, ordered by ascending p-value. If the * number of features is less than numTopFeatures, then this will select all features. - * Only applicable when selectorType = "kbest". + * Only applicable when selectorType = "numTopFeatures". * The default value of numTopFeatures is 50. * * @group param */ + @Since("1.6.0") final val numTopFeatures = new IntParam(this, "numTopFeatures", - "Number of features that selector will select, ordered by statistics value descending. If the" + + "Number of features that selector will select, ordered by ascending p-value. If the" + " number of features is < numTopFeatures, then this will select all features.", ParamValidators.gtEq(1)) setDefault(numTopFeatures -> 50) /** @group getParam */ + @Since("1.6.0") def getNumTopFeatures: Int = $(numTopFeatures) /** * Percentile of features that selector will select, ordered by statistics value descending. * Only applicable when selectorType = "percentile". * Default value is 0.1. + * @group param */ + @Since("2.1.0") final val percentile = new DoubleParam(this, "percentile", - "Percentile of features that selector will select, ordered by statistics value descending.", + "Percentile of features that selector will select, ordered by ascending p-value.", ParamValidators.inRange(0, 1)) setDefault(percentile -> 0.1) /** @group getParam */ + @Since("2.1.0") def getPercentile: Double = $(percentile) /** * The highest p-value for features to be kept. * Only applicable when selectorType = "fpr". * Default value is 0.05. + * @group param */ - final val alpha = new DoubleParam(this, "alpha", "The highest p-value for features to be kept.", + final val fpr = new DoubleParam(this, "fpr", "The highest p-value for features to be kept.", ParamValidators.inRange(0, 1)) - setDefault(alpha -> 0.05) + setDefault(fpr -> 0.05) /** @group getParam */ - def getAlpha: Double = $(alpha) + def getFpr: Double = $(fpr) /** * The selector type of the ChisqSelector. - * Supported options: "kbest" (default), "percentile" and "fpr". + * Supported options: "numTopFeatures" (default), "percentile", "fpr". + * @group param */ + @Since("2.1.0") final val selectorType = new Param[String](this, "selectorType", "The selector type of the ChisqSelector. " + - "Supported options: kbest (default), percentile and fpr.", - ParamValidators.inArray[String](OldChiSqSelector.supportedSelectorTypes.toArray)) - setDefault(selectorType -> OldChiSqSelector.KBest) + "Supported options: " + OldChiSqSelector.supportedSelectorTypes.mkString(", "), + ParamValidators.inArray[String](OldChiSqSelector.supportedSelectorTypes)) + setDefault(selectorType -> OldChiSqSelector.NumTopFeatures) /** @group getParam */ + @Since("2.1.0") def getSelectorType: String = $(selectorType) } /** * Chi-Squared feature selection, which selects categorical features to use for predicting a * categorical label. - * The selector supports three selection methods: `kbest`, `percentile` and `fpr`. - * `kbest` chooses the `k` top features according to a chi-squared test. - * `percentile` is similar but chooses a fraction of all features instead of a fixed number. - * `fpr` chooses all features whose false positive rate meets some threshold. - * By default, the selection method is `kbest`, the default number of top features is 50. + * The selector supports different selection methods: `numTopFeatures`, `percentile`, `fpr`. + * - `numTopFeatures` chooses a fixed number of top features according to a chi-squared test. + * - `percentile` is similar but chooses a fraction of all features instead of a fixed number. + * - `fpr` chooses all features whose p-value is below a threshold, thus controlling the false + * positive rate of selection. + * By default, the selection method is `numTopFeatures`, with the default number of top features + * set to 50. */ @Since("1.6.0") final class ChiSqSelector @Since("1.6.0") (@Since("1.6.0") override val uid: String) @@ -113,10 +124,6 @@ final class ChiSqSelector @Since("1.6.0") (@Since("1.6.0") override val uid: Str @Since("1.6.0") def this() = this(Identifiable.randomUID("chiSqSelector")) - /** @group setParam */ - @Since("2.1.0") - def setSelectorType(value: String): this.type = set(selectorType, value) - /** @group setParam */ @Since("1.6.0") def setNumTopFeatures(value: Int): this.type = set(numTopFeatures, value) @@ -127,7 +134,11 @@ final class ChiSqSelector @Since("1.6.0") (@Since("1.6.0") override val uid: Str /** @group setParam */ @Since("2.1.0") - def setAlpha(value: Double): this.type = set(alpha, value) + def setFpr(value: Double): this.type = set(fpr, value) + + /** @group setParam */ + @Since("2.1.0") + def setSelectorType(value: String): this.type = set(selectorType, value) /** @group setParam */ @Since("1.6.0") @@ -153,15 +164,15 @@ final class ChiSqSelector @Since("1.6.0") (@Since("1.6.0") override val uid: Str .setSelectorType($(selectorType)) .setNumTopFeatures($(numTopFeatures)) .setPercentile($(percentile)) - .setAlpha($(alpha)) + .setFpr($(fpr)) val model = selector.fit(input) copyValues(new ChiSqSelectorModel(uid, model).setParent(this)) } @Since("1.6.0") override def transformSchema(schema: StructType): StructType = { - val otherPairs = OldChiSqSelector.supportedTypeAndParamPairs.filter(_._1 != $(selectorType)) - otherPairs.foreach { case (_, paramName: String) => + val otherPairs = OldChiSqSelector.supportedSelectorTypes.filter(_ != $(selectorType)) + otherPairs.foreach { paramName: String => if (isSet(getParam(paramName))) { logWarning(s"Param $paramName will take no effect when selector type = ${$(selectorType)}.") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 904000f50d0a..034e3625e8c0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -638,13 +638,13 @@ private[python] class PythonMLLibAPI extends Serializable { selectorType: String, numTopFeatures: Int, percentile: Double, - alpha: Double, + fpr: Double, data: JavaRDD[LabeledPoint]): ChiSqSelectorModel = { new ChiSqSelector() .setSelectorType(selectorType) .setNumTopFeatures(numTopFeatures) .setPercentile(percentile) - .setAlpha(alpha) + .setFpr(fpr) .fit(data.rdd) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala index f8276de4f23d..f9156b642785 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala @@ -161,7 +161,7 @@ object ChiSqSelectorModel extends Loader[ChiSqSelectorModel] { Loader.checkSchema[Data](dataFrame.schema) val features = dataArray.rdd.map { - case Row(feature: Int) => (feature) + case Row(feature: Int) => feature }.collect() new ChiSqSelectorModel(features) @@ -171,18 +171,20 @@ object ChiSqSelectorModel extends Loader[ChiSqSelectorModel] { /** * Creates a ChiSquared feature selector. - * The selector supports three selection methods: `kbest`, `percentile` and `fpr`. - * `kbest` chooses the `k` top features according to a chi-squared test. - * `percentile` is similar but chooses a fraction of all features instead of a fixed number. - * `fpr` chooses all features whose false positive rate meets some threshold. - * By default, the selection method is `kbest`, the default number of top features is 50. + * The selector supports different selection methods: `numTopFeatures`, `percentile`, `fpr`. + * - `numTopFeatures` chooses a fixed number of top features according to a chi-squared test. + * - `percentile` is similar but chooses a fraction of all features instead of a fixed number. + * - `fpr` chooses all features whose p-value is below a threshold, thus controlling the false + * positive rate of selection. + * By default, the selection method is `numTopFeatures`, with the default number of top features + * set to 50. */ @Since("1.3.0") class ChiSqSelector @Since("2.1.0") () extends Serializable { var numTopFeatures: Int = 50 var percentile: Double = 0.1 - var alpha: Double = 0.05 - var selectorType = ChiSqSelector.KBest + var fpr: Double = 0.05 + var selectorType = ChiSqSelector.NumTopFeatures /** * The is the same to call this() and setNumTopFeatures(numTopFeatures) @@ -207,15 +209,15 @@ class ChiSqSelector @Since("2.1.0") () extends Serializable { } @Since("2.1.0") - def setAlpha(value: Double): this.type = { - require(0.0 <= value && value <= 1.0, "Alpha must be in [0,1]") - alpha = value + def setFpr(value: Double): this.type = { + require(0.0 <= value && value <= 1.0, "FPR must be in [0,1]") + fpr = value this } @Since("2.1.0") def setSelectorType(value: String): this.type = { - require(ChiSqSelector.supportedSelectorTypes.toSeq.contains(value), + require(ChiSqSelector.supportedSelectorTypes.contains(value), s"ChiSqSelector Type: $value was not supported.") selectorType = value this @@ -232,7 +234,7 @@ class ChiSqSelector @Since("2.1.0") () extends Serializable { def fit(data: RDD[LabeledPoint]): ChiSqSelectorModel = { val chiSqTestResult = Statistics.chiSqTest(data).zipWithIndex val features = selectorType match { - case ChiSqSelector.KBest => + case ChiSqSelector.NumTopFeatures => chiSqTestResult .sortBy { case (res, _) => res.pValue } .take(numTopFeatures) @@ -242,7 +244,7 @@ class ChiSqSelector @Since("2.1.0") () extends Serializable { .take((chiSqTestResult.length * percentile).toInt) case ChiSqSelector.FPR => chiSqTestResult - .filter { case (res, _) => res.pValue < alpha } + .filter { case (res, _) => res.pValue < fpr } case errorType => throw new IllegalStateException(s"Unknown ChiSqSelector Type: $errorType") } @@ -251,22 +253,17 @@ class ChiSqSelector @Since("2.1.0") () extends Serializable { } } -@Since("2.1.0") -object ChiSqSelector { +private[spark] object ChiSqSelector { - /** String name for `kbest` selector type. */ - private[spark] val KBest: String = "kbest" + /** String name for `numTopFeatures` selector type. */ + val NumTopFeatures: String = "numTopFeatures" /** String name for `percentile` selector type. */ - private[spark] val Percentile: String = "percentile" + val Percentile: String = "percentile" /** String name for `fpr` selector type. */ private[spark] val FPR: String = "fpr" - /** Set of selector type and param pairs that ChiSqSelector supports. */ - private[spark] val supportedTypeAndParamPairs = Set(KBest -> "numTopFeatures", - Percentile -> "percentile", FPR -> "alpha") - /** Set of selector types that ChiSqSelector supports. */ - private[spark] val supportedSelectorTypes = supportedTypeAndParamPairs.map(_._1) + val supportedSelectorTypes: Array[String] = Array(NumTopFeatures, Percentile, FPR) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala index 6af06d82d671..80970fd74488 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/ChiSqSelectorSuite.scala @@ -19,85 +19,72 @@ package org.apache.spark.ml.feature import org.apache.spark.SparkFunSuite import org.apache.spark.ml.linalg.{Vector, Vectors} +import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ -import org.apache.spark.mllib.feature import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.Row +import org.apache.spark.sql.{Dataset, Row} class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { - test("Test Chi-Square selector") { - import testImplicits._ - val data = Seq( - LabeledPoint(0.0, Vectors.sparse(3, Array((0, 8.0), (1, 7.0)))), - LabeledPoint(1.0, Vectors.sparse(3, Array((1, 9.0), (2, 6.0)))), - LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 8.0))), - LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 5.0))) - ) + @transient var dataset: Dataset[_] = _ - val preFilteredData = Seq( - Vectors.dense(8.0), - Vectors.dense(0.0), - Vectors.dense(0.0), - Vectors.dense(8.0) - ) + override def beforeAll(): Unit = { + super.beforeAll() - val df = sc.parallelize(data.zip(preFilteredData)) - .map(x => (x._1.label, x._1.features, x._2)) - .toDF("label", "data", "preFilteredData") - - val selector = new ChiSqSelector() - .setSelectorType("kbest") - .setNumTopFeatures(1) - .setFeaturesCol("data") - .setLabelCol("label") - .setOutputCol("filtered") - - selector.fit(df).transform(df).select("filtered", "preFilteredData").collect().foreach { - case Row(vec1: Vector, vec2: Vector) => - assert(vec1 ~== vec2 absTol 1e-1) - } - - selector.setSelectorType("percentile").setPercentile(0.34).fit(df).transform(df) - .select("filtered", "preFilteredData").collect().foreach { - case Row(vec1: Vector, vec2: Vector) => - assert(vec1 ~== vec2 absTol 1e-1) - } + // Toy dataset, including the top feature for a chi-squared test. + // These data are chosen such that each feature's test has a distinct p-value. + /* To verify the results with R, run: + library(stats) + x1 <- c(8.0, 0.0, 0.0, 7.0, 8.0) + x2 <- c(7.0, 9.0, 9.0, 9.0, 7.0) + x3 <- c(0.0, 6.0, 8.0, 5.0, 3.0) + y <- c(0.0, 1.0, 1.0, 2.0, 2.0) + chisq.test(x1,y) + chisq.test(x2,y) + chisq.test(x3,y) + */ + dataset = spark.createDataFrame(Seq( + (0.0, Vectors.sparse(3, Array((0, 8.0), (1, 7.0))), Vectors.dense(8.0)), + (1.0, Vectors.sparse(3, Array((1, 9.0), (2, 6.0))), Vectors.dense(0.0)), + (1.0, Vectors.dense(Array(0.0, 9.0, 8.0)), Vectors.dense(0.0)), + (2.0, Vectors.dense(Array(7.0, 9.0, 5.0)), Vectors.dense(7.0)), + (2.0, Vectors.dense(Array(8.0, 7.0, 3.0)), Vectors.dense(8.0)) + )).toDF("label", "features", "topFeature") + } - val preFilteredData2 = Seq( - Vectors.dense(8.0, 7.0), - Vectors.dense(0.0, 9.0), - Vectors.dense(0.0, 9.0), - Vectors.dense(8.0, 9.0) - ) + test("params") { + ParamsSuite.checkParams(new ChiSqSelector) + val model = new ChiSqSelectorModel("myModel", + new org.apache.spark.mllib.feature.ChiSqSelectorModel(Array(1, 3, 4))) + ParamsSuite.checkParams(model) + } - val df2 = sc.parallelize(data.zip(preFilteredData2)) - .map(x => (x._1.label, x._1.features, x._2)) - .toDF("label", "data", "preFilteredData") + test("Test Chi-Square selector: numTopFeatures") { + val selector = new ChiSqSelector() + .setOutputCol("filtered").setSelectorType("numTopFeatures").setNumTopFeatures(1) + ChiSqSelectorSuite.testSelector(selector, dataset) + } - selector.setSelectorType("fpr").setAlpha(0.2).fit(df2).transform(df2) - .select("filtered", "preFilteredData").collect().foreach { - case Row(vec1: Vector, vec2: Vector) => - assert(vec1 ~== vec2 absTol 1e-1) - } + test("Test Chi-Square selector: percentile") { + val selector = new ChiSqSelector() + .setOutputCol("filtered").setSelectorType("percentile").setPercentile(0.34) + ChiSqSelectorSuite.testSelector(selector, dataset) } - test("ChiSqSelector read/write") { - val t = new ChiSqSelector() - .setFeaturesCol("myFeaturesCol") - .setLabelCol("myLabelCol") - .setOutputCol("myOutputCol") - .setNumTopFeatures(2) - testDefaultReadWrite(t) + test("Test Chi-Square selector: fpr") { + val selector = new ChiSqSelector() + .setOutputCol("filtered").setSelectorType("fpr").setFpr(0.2) + ChiSqSelectorSuite.testSelector(selector, dataset) } - test("ChiSqSelectorModel read/write") { - val oldModel = new feature.ChiSqSelectorModel(Array(1, 3)) - val instance = new ChiSqSelectorModel("myChiSqSelectorModel", oldModel) - val newInstance = testDefaultReadWrite(instance) - assert(newInstance.selectedFeatures === instance.selectedFeatures) + test("read/write") { + def checkModelData(model: ChiSqSelectorModel, model2: ChiSqSelectorModel): Unit = { + assert(model.selectedFeatures === model2.selectedFeatures) + } + val nb = new ChiSqSelector + testEstimatorAndModelReadWrite(nb, dataset, ChiSqSelectorSuite.allParamSettings, checkModelData) } test("should support all NumericType labels and not support other types") { @@ -108,3 +95,25 @@ class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext } } } + +object ChiSqSelectorSuite { + + private def testSelector(selector: ChiSqSelector, dataset: Dataset[_]): Unit = { + selector.fit(dataset).transform(dataset).select("filtered", "topFeature").collect() + .foreach { case Row(vec1: Vector, vec2: Vector) => + assert(vec1 ~== vec2 absTol 1e-1) + } + } + + /** + * Mapping from all Params to valid settings which differ from the defaults. + * This is useful for tests which need to exercise all Params, such as save/load. + * This excludes input columns to simplify some tests. + */ + val allParamSettings: Map[String, Any] = Map( + "selectorType" -> "percentile", + "numTopFeatures" -> 1, + "percentile" -> 0.12, + "outputCol" -> "myOutput" + ) +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala index ac702b4b7c69..77219e500617 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/ChiSqSelectorSuite.scala @@ -54,33 +54,34 @@ class ChiSqSelectorSuite extends SparkFunSuite with MLlibTestSparkContext { LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 8.0))), LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 5.0)))), 2) val preFilteredData = - Set(LabeledPoint(0.0, Vectors.dense(Array(8.0))), + Seq(LabeledPoint(0.0, Vectors.dense(Array(8.0))), LabeledPoint(1.0, Vectors.dense(Array(0.0))), LabeledPoint(1.0, Vectors.dense(Array(0.0))), LabeledPoint(2.0, Vectors.dense(Array(8.0)))) val model = new ChiSqSelector(1).fit(labeledDiscreteData) val filteredData = labeledDiscreteData.map { lp => LabeledPoint(lp.label, model.transform(lp.features)) - }.collect().toSet - assert(filteredData == preFilteredData) + }.collect().toSeq + assert(filteredData === preFilteredData) } - test("ChiSqSelector by FPR transform test (sparse & dense vector)") { + test("ChiSqSelector by fpr transform test (sparse & dense vector)") { val labeledDiscreteData = sc.parallelize( Seq(LabeledPoint(0.0, Vectors.sparse(4, Array((0, 8.0), (1, 7.0)))), LabeledPoint(1.0, Vectors.sparse(4, Array((1, 9.0), (2, 6.0), (3, 4.0)))), LabeledPoint(1.0, Vectors.dense(Array(0.0, 9.0, 8.0, 4.0))), LabeledPoint(2.0, Vectors.dense(Array(8.0, 9.0, 5.0, 9.0)))), 2) val preFilteredData = - Set(LabeledPoint(0.0, Vectors.dense(Array(0.0))), + Seq(LabeledPoint(0.0, Vectors.dense(Array(0.0))), LabeledPoint(1.0, Vectors.dense(Array(4.0))), LabeledPoint(1.0, Vectors.dense(Array(4.0))), LabeledPoint(2.0, Vectors.dense(Array(9.0)))) - val model = new ChiSqSelector().setSelectorType("fpr").setAlpha(0.1).fit(labeledDiscreteData) + val model: ChiSqSelectorModel = new ChiSqSelector().setSelectorType("fpr") + .setFpr(0.1).fit(labeledDiscreteData) val filteredData = labeledDiscreteData.map { lp => LabeledPoint(lp.label, model.transform(lp.features)) - }.collect().toSet - assert(filteredData == preFilteredData) + }.collect().toSeq + assert(filteredData === preFilteredData) } test("model load / save") { diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 94afe82a3647..635cf1304588 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -2606,42 +2606,43 @@ class ChiSqSelector(JavaEstimator, HasFeaturesCol, HasOutputCol, HasLabelCol, Ja selectorType = Param(Params._dummy(), "selectorType", "The selector type of the ChisqSelector. " + - "Supported options: kbest (default), percentile and fpr.", + "Supported options: numTopFeatures (default), percentile and fpr.", typeConverter=TypeConverters.toString) numTopFeatures = \ Param(Params._dummy(), "numTopFeatures", - "Number of features that selector will select, ordered by statistics value " + - "descending. If the number of features is < numTopFeatures, then this will select " + + "Number of features that selector will select, ordered by ascending p-value. " + + "If the number of features is < numTopFeatures, then this will select " + "all features.", typeConverter=TypeConverters.toInt) percentile = Param(Params._dummy(), "percentile", "Percentile of features that selector " + - "will select, ordered by statistics value descending.", + "will select, ordered by ascending p-value.", typeConverter=TypeConverters.toFloat) - alpha = Param(Params._dummy(), "alpha", "The highest p-value for features to be kept.", - typeConverter=TypeConverters.toFloat) + fpr = Param(Params._dummy(), "fpr", "The highest p-value for features to be kept.", + typeConverter=TypeConverters.toFloat) @keyword_only def __init__(self, numTopFeatures=50, featuresCol="features", outputCol=None, - labelCol="label", selectorType="kbest", percentile=0.1, alpha=0.05): + labelCol="label", selectorType="numTopFeatures", percentile=0.1, fpr=0.05): """ __init__(self, numTopFeatures=50, featuresCol="features", outputCol=None, \ - labelCol="label", selectorType="kbest", percentile=0.1, alpha=0.05) + labelCol="label", selectorType="numTopFeatures", percentile=0.1, fpr=0.05) """ super(ChiSqSelector, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.feature.ChiSqSelector", self.uid) - self._setDefault(numTopFeatures=50, selectorType="kbest", percentile=0.1, alpha=0.05) + self._setDefault(numTopFeatures=50, selectorType="numTopFeatures", percentile=0.1, + fpr=0.05) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only @since("2.0.0") def setParams(self, numTopFeatures=50, featuresCol="features", outputCol=None, - labelCol="labels", selectorType="kbest", percentile=0.1, alpha=0.05): + labelCol="labels", selectorType="numTopFeatures", percentile=0.1, fpr=0.05): """ setParams(self, numTopFeatures=50, featuresCol="features", outputCol=None, \ - labelCol="labels", selectorType="kbest", percentile=0.1, alpha=0.05) + labelCol="labels", selectorType="numTopFeatures", percentile=0.1, fpr=0.05) Sets params for this ChiSqSelector. """ kwargs = self.setParams._input_kwargs @@ -2665,7 +2666,7 @@ def getSelectorType(self): def setNumTopFeatures(self, value): """ Sets the value of :py:attr:`numTopFeatures`. - Only applicable when selectorType = "kbest". + Only applicable when selectorType = "numTopFeatures". """ return self._set(numTopFeatures=value) @@ -2692,19 +2693,19 @@ def getPercentile(self): return self.getOrDefault(self.percentile) @since("2.1.0") - def setAlpha(self, value): + def setFpr(self, value): """ - Sets the value of :py:attr:`alpha`. + Sets the value of :py:attr:`fpr`. Only applicable when selectorType = "fpr". """ - return self._set(alpha=value) + return self._set(fpr=value) @since("2.1.0") - def getAlpha(self): + def getFpr(self): """ - Gets the value of alpha or its default value. + Gets the value of fpr or its default value. """ - return self.getOrDefault(self.alpha) + return self.getOrDefault(self.fpr) def _create_model(self, java_model): return ChiSqSelectorModel(java_model) diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 50ef7c7901c2..7eaa2282cb8b 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -274,52 +274,48 @@ def transform(self, vector): class ChiSqSelector(object): """ Creates a ChiSquared feature selector. - The selector supports three selection methods: `KBest`, `Percentile` and `FPR`. - `kbest` chooses the `k` top features according to a chi-squared test. + The selector supports different selection methods: `numTopFeatures`, `percentile`, `fpr`. + `numTopFeatures` chooses a fixed number of top features according to a chi-squared test. `percentile` is similar but chooses a fraction of all features instead of a fixed number. - `fpr` chooses all features whose false positive rate meets some threshold. - By default, the selection method is `kbest`, the default number of top features is 50. + `fpr` chooses all features whose p-value is below a threshold, thus controlling the false + positive rate of selection. + By default, the selection method is `numTopFeatures`, with the default number of top features + set to 50. - >>> data = [ + >>> data = sc.parallelize([ ... LabeledPoint(0.0, SparseVector(3, {0: 8.0, 1: 7.0})), ... LabeledPoint(1.0, SparseVector(3, {1: 9.0, 2: 6.0})), ... LabeledPoint(1.0, [0.0, 9.0, 8.0]), - ... LabeledPoint(2.0, [8.0, 9.0, 5.0]) - ... ] - >>> model = ChiSqSelector().setNumTopFeatures(1).fit(sc.parallelize(data)) + ... LabeledPoint(2.0, [7.0, 9.0, 5.0]), + ... LabeledPoint(2.0, [8.0, 7.0, 3.0]) + ... ]) + >>> model = ChiSqSelector(numTopFeatures=1).fit(data) >>> model.transform(SparseVector(3, {1: 9.0, 2: 6.0})) SparseVector(1, {}) - >>> model.transform(DenseVector([8.0, 9.0, 5.0])) - DenseVector([8.0]) - >>> model = ChiSqSelector().setSelectorType("percentile").setPercentile(0.34).fit( - ... sc.parallelize(data)) + >>> model.transform(DenseVector([7.0, 9.0, 5.0])) + DenseVector([7.0]) + >>> model = ChiSqSelector(selectorType="fpr", fpr=0.2).fit(data) >>> model.transform(SparseVector(3, {1: 9.0, 2: 6.0})) SparseVector(1, {}) - >>> model.transform(DenseVector([8.0, 9.0, 5.0])) - DenseVector([8.0]) - >>> data = [ - ... LabeledPoint(0.0, SparseVector(4, {0: 8.0, 1: 7.0})), - ... LabeledPoint(1.0, SparseVector(4, {1: 9.0, 2: 6.0, 3: 4.0})), - ... LabeledPoint(1.0, [0.0, 9.0, 8.0, 4.0]), - ... LabeledPoint(2.0, [8.0, 9.0, 5.0, 9.0]) - ... ] - >>> model = ChiSqSelector().setSelectorType("fpr").setAlpha(0.1).fit(sc.parallelize(data)) - >>> model.transform(DenseVector([1.0,2.0,3.0,4.0])) - DenseVector([4.0]) + >>> model.transform(DenseVector([7.0, 9.0, 5.0])) + DenseVector([7.0]) + >>> model = ChiSqSelector(selectorType="percentile", percentile=0.34).fit(data) + >>> model.transform(DenseVector([7.0, 9.0, 5.0])) + DenseVector([7.0]) .. versionadded:: 1.4.0 """ - def __init__(self, numTopFeatures=50, selectorType="kbest", percentile=0.1, alpha=0.05): + def __init__(self, numTopFeatures=50, selectorType="numTopFeatures", percentile=0.1, fpr=0.05): self.numTopFeatures = numTopFeatures self.selectorType = selectorType self.percentile = percentile - self.alpha = alpha + self.fpr = fpr @since('2.1.0') def setNumTopFeatures(self, numTopFeatures): """ set numTopFeature for feature selection by number of top features. - Only applicable when selectorType = "kbest". + Only applicable when selectorType = "numTopFeatures". """ self.numTopFeatures = int(numTopFeatures) return self @@ -334,19 +330,19 @@ def setPercentile(self, percentile): return self @since('2.1.0') - def setAlpha(self, alpha): + def setFpr(self, fpr): """ - set alpha [0.0, 1.0] for feature selection by FPR. + set FPR [0.0, 1.0] for feature selection by FPR. Only applicable when selectorType = "fpr". """ - self.alpha = float(alpha) + self.fpr = float(fpr) return self @since('2.1.0') def setSelectorType(self, selectorType): """ set the selector type of the ChisqSelector. - Supported options: "kbest" (default), "percentile" and "fpr". + Supported options: "numTopFeatures" (default), "percentile", "fpr". """ self.selectorType = str(selectorType) return self @@ -362,7 +358,7 @@ def fit(self, data): Apply feature discretizer before using this function. """ jmodel = callMLlibFunc("fitChiSqSelector", self.selectorType, self.numTopFeatures, - self.percentile, self.alpha, data) + self.percentile, self.fpr, data) return ChiSqSelectorModel(jmodel) From 77a98162d1ec28247053b8b3ad4af28baa950797 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 1 Nov 2016 18:06:57 -0700 Subject: [PATCH 034/132] [SPARK-18025] Use commit protocol API in structured streaming ## What changes were proposed in this pull request? This patch adds a new commit protocol implementation ManifestFileCommitProtocol that follows the existing streaming flow, and uses it in FileStreamSink to consolidate the write path in structured streaming with the batch mode write path. This deletes a lot of code, and would make it trivial to support other functionalities that are currently available in batch but not in streaming, including all file formats and bucketing. ## How was this patch tested? Should be covered by existing tests. Author: Reynold Xin Closes #15710 from rxin/SPARK-18025. --- .../datasources/FileCommitProtocol.scala | 11 +- .../execution/datasources/FileFormat.scala | 14 -- ...iteOutput.scala => FileFormatWriter.scala} | 20 +- .../InsertIntoHadoopFsRelationCommand.scala | 25 +- .../parquet/ParquetFileFormat.scala | 11 - .../parquet/ParquetOutputWriter.scala | 116 +-------- .../execution/streaming/FileStreamSink.scala | 229 ++---------------- .../ManifestFileCommitProtocol.scala | 114 +++++++++ .../apache/spark/sql/internal/SQLConf.scala | 3 +- .../sql/streaming/FileStreamSinkSuite.scala | 106 +------- 10 files changed, 174 insertions(+), 475 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/{WriteOutput.scala => FileFormatWriter.scala} (97%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCommitProtocol.scala index 1ce9ae4266c1..f5dd5ce22919 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCommitProtocol.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileCommitProtocol.scala @@ -32,9 +32,9 @@ import org.apache.spark.util.Utils object FileCommitProtocol { - class TaskCommitMessage(obj: Any) extends Serializable + class TaskCommitMessage(val obj: Any) extends Serializable - object EmptyTaskCommitMessage extends TaskCommitMessage(Unit) + object EmptyTaskCommitMessage extends TaskCommitMessage(null) /** * Instantiates a FileCommitProtocol using the given className. @@ -62,8 +62,11 @@ object FileCommitProtocol { /** - * An interface to define how a Spark job commits its outputs. Implementations must be serializable, - * as the committer instance instantiated on the driver will be used for tasks on executors. + * An interface to define how a single Spark job commits its outputs. Two notes: + * + * 1. Implementations must be serializable, as the committer instance instantiated on the driver + * will be used for tasks on executors. + * 2. A committer should not be reused across multiple Spark jobs. * * The proper call sequence is: * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala index 9d153cec731a..4f4aaaa5026f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala @@ -55,20 +55,6 @@ trait FileFormat { options: Map[String, String], dataSchema: StructType): OutputWriterFactory - /** - * Returns a [[OutputWriterFactory]] for generating output writers that can write data. - * This method is current used only by FileStreamSinkWriter to generate output writers that - * does not use output committers to write data. The OutputWriter generated by the returned - * [[OutputWriterFactory]] must implement the method `newWriter(path)`.. - */ - def buildWriter( - sqlContext: SQLContext, - dataSchema: StructType, - options: Map[String, String]): OutputWriterFactory = { - // TODO: Remove this default implementation when the other formats have been ported - throw new UnsupportedOperationException(s"buildWriter is not supported for $this") - } - /** * Returns whether this format support returning columnar batch or not. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala similarity index 97% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index a07855111b40..bc00a0a749c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -43,8 +43,8 @@ import org.apache.spark.util.{SerializableConfiguration, Utils} import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter -/** A helper object for writing data out to a location. */ -object WriteOutput extends Logging { +/** A helper object for writing FileFormat data out to a location. */ +object FileFormatWriter extends Logging { /** A shared job description for all the write tasks. */ private class WriteJobDescription( @@ -55,7 +55,6 @@ object WriteOutput extends Logging { val partitionColumns: Seq[Attribute], val nonPartitionColumns: Seq[Attribute], val bucketSpec: Option[BucketSpec], - val isAppend: Boolean, val path: String) extends Serializable { @@ -82,18 +81,18 @@ object WriteOutput extends Logging { sparkSession: SparkSession, plan: LogicalPlan, fileFormat: FileFormat, - outputPath: Path, + committer: FileCommitProtocol, + outputPath: String, hadoopConf: Configuration, partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], refreshFunction: (Seq[TablePartitionSpec]) => Unit, - options: Map[String, String], - isAppend: Boolean): Unit = { + options: Map[String, String]): Unit = { val job = Job.getInstance(hadoopConf) job.setOutputKeyClass(classOf[Void]) job.setOutputValueClass(classOf[InternalRow]) - FileOutputFormat.setOutputPath(job, outputPath) + FileOutputFormat.setOutputPath(job, new Path(outputPath)) val partitionSet = AttributeSet(partitionColumns) val dataColumns = plan.output.filterNot(partitionSet.contains) @@ -111,16 +110,11 @@ object WriteOutput extends Logging { partitionColumns = partitionColumns, nonPartitionColumns = dataColumns, bucketSpec = bucketSpec, - isAppend = isAppend, - path = outputPath.toString) + path = outputPath) SQLExecution.withNewExecutionId(sparkSession, queryExecution) { // This call shouldn't be put into the `try` block below because it only initializes and // prepares the job, any exception thrown from here shouldn't cause abortJob() to be called. - val committer = FileCommitProtocol.instantiate( - sparkSession.sessionState.conf.fileCommitProtocolClass, - outputPath.toString, - isAppend) committer.setupJob(job) try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index a1221d0ae6d2..230c74a47ba2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -84,17 +84,22 @@ case class InsertIntoHadoopFsRelationCommand( val isAppend = pathExists && (mode == SaveMode.Append) if (doInsertion) { - WriteOutput.write( - sparkSession, - query, - fileFormat, - qualifiedOutputPath, - hadoopConf, - partitionColumns, - bucketSpec, - refreshFunction, - options, + val committer = FileCommitProtocol.instantiate( + sparkSession.sessionState.conf.fileCommitProtocolClass, + outputPath.toString, isAppend) + + FileFormatWriter.write( + sparkSession = sparkSession, + plan = query, + fileFormat = fileFormat, + committer = committer, + outputPath = qualifiedOutputPath.toString, + hadoopConf = hadoopConf, + partitionColumns = partitionColumns, + bucketSpec = bucketSpec, + refreshFunction = refreshFunction, + options = options) } else { logInfo("Skipping insertion into a relation that already exists.") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 77c83ba38efe..b8ea7f40c4ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -415,17 +415,6 @@ class ParquetFileFormat } } } - - override def buildWriter( - sqlContext: SQLContext, - dataSchema: StructType, - options: Map[String, String]): OutputWriterFactory = { - new ParquetOutputWriterFactory( - sqlContext.conf, - dataSchema, - sqlContext.sessionState.newHadoopConf(), - options) - } } object ParquetFileFormat extends Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala index 92d4f27be3fd..5c0f8af17a23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala @@ -17,125 +17,13 @@ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl -import org.apache.parquet.hadoop.{ParquetOutputFormat, ParquetRecordWriter} -import org.apache.parquet.hadoop.codec.CodecConfig -import org.apache.parquet.hadoop.util.ContextUtil +import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.StructType -import org.apache.spark.util.SerializableConfiguration - - -/** - * A factory for generating OutputWriters for writing parquet files. This implemented is different - * from the [[ParquetOutputWriter]] as this does not use any [[OutputCommitter]]. It simply - * writes the data to the path used to generate the output writer. Callers of this factory - * has to ensure which files are to be considered as committed. - */ -private[parquet] class ParquetOutputWriterFactory( - sqlConf: SQLConf, - dataSchema: StructType, - hadoopConf: Configuration, - options: Map[String, String]) - extends OutputWriterFactory { - - private val serializableConf: SerializableConfiguration = { - val job = Job.getInstance(hadoopConf) - val conf = ContextUtil.getConfiguration(job) - val parquetOptions = new ParquetOptions(options, sqlConf) - - // We're not really using `ParquetOutputFormat[Row]` for writing data here, because we override - // it in `ParquetOutputWriter` to support appending and dynamic partitioning. The reason why - // we set it here is to setup the output committer class to `ParquetOutputCommitter`, which is - // bundled with `ParquetOutputFormat[Row]`. - job.setOutputFormatClass(classOf[ParquetOutputFormat[Row]]) - - ParquetOutputFormat.setWriteSupportClass(job, classOf[ParquetWriteSupport]) - - // We want to clear this temporary metadata from saving into Parquet file. - // This metadata is only useful for detecting optional columns when pushing down filters. - val dataSchemaToWrite = StructType.removeMetadata( - StructType.metadataKeyForOptionalField, - dataSchema).asInstanceOf[StructType] - ParquetWriteSupport.setSchema(dataSchemaToWrite, conf) - - // Sets flags for `CatalystSchemaConverter` (which converts Catalyst schema to Parquet schema) - // and `CatalystWriteSupport` (writing actual rows to Parquet files). - conf.set( - SQLConf.PARQUET_BINARY_AS_STRING.key, - sqlConf.isParquetBinaryAsString.toString) - - conf.set( - SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, - sqlConf.isParquetINT96AsTimestamp.toString) - - conf.set( - SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key, - sqlConf.writeLegacyParquetFormat.toString) - - // Sets compression scheme - conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodecClassName) - new SerializableConfiguration(conf) - } - - /** - * Returns a [[OutputWriter]] that writes data to the give path without using - * [[OutputCommitter]]. - */ - override def newWriter(path: String): OutputWriter = new OutputWriter { - - // Create TaskAttemptContext that is used to pass on Configuration to the ParquetRecordWriter - private val hadoopTaskAttemptId = new TaskAttemptID(new TaskID(new JobID, TaskType.MAP, 0), 0) - private val hadoopAttemptContext = new TaskAttemptContextImpl( - serializableConf.value, hadoopTaskAttemptId) - - // Instance of ParquetRecordWriter that does not use OutputCommitter - private val recordWriter = createNoCommitterRecordWriter(path, hadoopAttemptContext) - - override def write(row: Row): Unit = { - throw new UnsupportedOperationException("call writeInternal") - } - - protected[sql] override def writeInternal(row: InternalRow): Unit = { - recordWriter.write(null, row) - } - - override def close(): Unit = recordWriter.close(hadoopAttemptContext) - } - - /** Create a [[ParquetRecordWriter]] that writes the given path without using OutputCommitter */ - private def createNoCommitterRecordWriter( - path: String, - hadoopAttemptContext: TaskAttemptContext): RecordWriter[Void, InternalRow] = { - // Custom ParquetOutputFormat that disable use of committer and writes to the given path - val outputFormat = new ParquetOutputFormat[InternalRow]() { - override def getOutputCommitter(c: TaskAttemptContext): OutputCommitter = { null } - override def getDefaultWorkFile(c: TaskAttemptContext, ext: String): Path = { new Path(path) } - } - outputFormat.getRecordWriter(hadoopAttemptContext) - } - - /** Disable the use of the older API. */ - override def newInstance( - path: String, - dataSchema: StructType, - context: TaskAttemptContext): OutputWriter = { - throw new UnsupportedOperationException("this version of newInstance not supported for " + - "ParquetOutputWriterFactory") - } - - override def getFileExtension(context: TaskAttemptContext): String = { - CodecConfig.from(context).getCodec.getExtension + ".parquet" - } -} - +import org.apache.spark.sql.execution.datasources.OutputWriter // NOTE: This class is instantiated and used on executor side only, no need to be serializable. private[parquet] class ParquetOutputWriter(path: String, context: TaskAttemptContext) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala index 02c5b857ee7f..daec2b545097 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala @@ -17,23 +17,12 @@ package org.apache.spark.sql.execution.streaming -import java.util.UUID - -import scala.collection.mutable.ArrayBuffer - -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.{SparkEnv, SparkException, TaskContext, TaskContextImpl} import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.UnsafeKVExternalSorter -import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, PartitioningUtils} -import org.apache.spark.sql.types.{StringType, StructType} -import org.apache.spark.util.SerializableConfiguration -import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter +import org.apache.spark.sql.execution.datasources.{FileCommitProtocol, FileFormat, FileFormatWriter} object FileStreamSink { // The name of the subdirectory that is used to store metadata about which files are valid. @@ -59,207 +48,41 @@ class FileStreamSink( private val fileLog = new FileStreamSinkLog(FileStreamSinkLog.VERSION, sparkSession, logPath.toUri.toString) private val hadoopConf = sparkSession.sessionState.newHadoopConf() - private val fs = basePath.getFileSystem(hadoopConf) override def addBatch(batchId: Long, data: DataFrame): Unit = { if (batchId <= fileLog.getLatest().map(_._1).getOrElse(-1L)) { logInfo(s"Skipping already committed batch $batchId") } else { - val writer = new FileStreamSinkWriter( - data, fileFormat, path, partitionColumnNames, hadoopConf, options) - val fileStatuses = writer.write() - if (fileLog.add(batchId, fileStatuses)) { - logInfo(s"Committed batch $batchId") - } else { - throw new IllegalStateException(s"Race while writing batch $batchId") + val committer = FileCommitProtocol.instantiate( + sparkSession.sessionState.conf.streamingFileCommitProtocolClass, path, isAppend = false) + committer match { + case manifestCommitter: ManifestFileCommitProtocol => + manifestCommitter.setupManifestOptions(fileLog, batchId) + case _ => // Do nothing } - } - } - - override def toString: String = s"FileSink[$path]" -} - - -/** - * Writes data given to a [[FileStreamSink]] to the given `basePath` in the given `fileFormat`, - * partitioned by the given `partitionColumnNames`. This writer always appends data to the - * directory if it already has data. - */ -class FileStreamSinkWriter( - data: DataFrame, - fileFormat: FileFormat, - basePath: String, - partitionColumnNames: Seq[String], - hadoopConf: Configuration, - options: Map[String, String]) extends Serializable with Logging { - - PartitioningUtils.validatePartitionColumn( - data.schema, partitionColumnNames, data.sqlContext.conf.caseSensitiveAnalysis) - - private val serializableConf = new SerializableConfiguration(hadoopConf) - private val dataSchema = data.schema - private val dataColumns = data.logicalPlan.output - - // Get the actual partition columns as attributes after matching them by name with - // the given columns names. - private val partitionColumns = partitionColumnNames.map { col => - val nameEquality = data.sparkSession.sessionState.conf.resolver - data.logicalPlan.output.find(f => nameEquality(f.name, col)).getOrElse { - throw new RuntimeException(s"Partition column $col not found in schema $dataSchema") - } - } - - // Columns that are to be written to the files. If there are partitioning columns, then - // those will not be written to the files. - private val writeColumns = { - val partitionSet = AttributeSet(partitionColumns) - dataColumns.filterNot(partitionSet.contains) - } - - // An OutputWriterFactory for generating writers in the executors for writing the files. - private val outputWriterFactory = - fileFormat.buildWriter(data.sqlContext, writeColumns.toStructType, options) - - /** Expressions that given a partition key build a string like: col1=val/col2=val/... */ - private def partitionStringExpression: Seq[Expression] = { - partitionColumns.zipWithIndex.flatMap { case (c, i) => - val escaped = - ScalaUDF( - PartitioningUtils.escapePathName _, - StringType, - Seq(Cast(c, StringType)), - Seq(StringType)) - val str = If(IsNull(c), Literal(PartitioningUtils.DEFAULT_PARTITION_NAME), escaped) - val partitionName = Literal(c.name + "=") :: str :: Nil - if (i == 0) partitionName else Literal(Path.SEPARATOR) :: partitionName - } - } - - /** Generate a new output writer from the writer factory */ - private def newOutputWriter(path: Path): OutputWriter = { - val newWriter = outputWriterFactory.newWriter(path.toString) - newWriter.initConverter(dataSchema) - newWriter - } - /** Write the dataframe to files. This gets called in the driver by the [[FileStreamSink]]. */ - def write(): Array[SinkFileStatus] = { - data.sqlContext.sparkContext.runJob( - data.queryExecution.toRdd, - (taskContext: TaskContext, iterator: Iterator[InternalRow]) => { - if (partitionColumns.isEmpty) { - Seq(writePartitionToSingleFile(iterator)) - } else { - writePartitionToPartitionedFiles(iterator) + // Get the actual partition columns as attributes after matching them by name with + // the given columns names. + val partitionColumns: Seq[Attribute] = partitionColumnNames.map { col => + val nameEquality = data.sparkSession.sessionState.conf.resolver + data.logicalPlan.output.find(f => nameEquality(f.name, col)).getOrElse { + throw new RuntimeException(s"Partition column $col not found in schema ${data.schema}") } - }).flatten - } - - /** - * Writes a RDD partition to a single file without dynamic partitioning. - * This gets called in the executor, and it uses a [[OutputWriter]] to write the data. - */ - def writePartitionToSingleFile(iterator: Iterator[InternalRow]): SinkFileStatus = { - var writer: OutputWriter = null - try { - val path = new Path(basePath, UUID.randomUUID.toString) - val fs = path.getFileSystem(serializableConf.value) - writer = newOutputWriter(path) - while (iterator.hasNext) { - writer.writeInternal(iterator.next) - } - writer.close() - writer = null - SinkFileStatus(fs.getFileStatus(path)) - } catch { - case cause: Throwable => - logError("Aborting task.", cause) - // call failure callbacks first, so we could have a chance to cleanup the writer. - TaskContext.get().asInstanceOf[TaskContextImpl].markTaskFailed(cause) - throw new SparkException("Task failed while writing rows.", cause) - } finally { - if (writer != null) { - writer.close() } - } - } - - /** - * Writes a RDD partition to multiple dynamically partitioned files. - * This gets called in the executor. It first sorts the data based on the partitioning columns - * and then writes the data of each key to separate files using [[OutputWriter]]s. - */ - def writePartitionToPartitionedFiles(iterator: Iterator[InternalRow]): Seq[SinkFileStatus] = { - - // Returns the partitioning columns for sorting - val getSortingKey = UnsafeProjection.create(partitionColumns, dataColumns) - - // Returns the data columns to be written given an input row - val getOutputRow = UnsafeProjection.create(writeColumns, dataColumns) - - // Returns the partition path given a partition key - val getPartitionString = - UnsafeProjection.create(Concat(partitionStringExpression) :: Nil, partitionColumns) - // Sort the data before write, so that we only need one writer at the same time. - val sorter = new UnsafeKVExternalSorter( - partitionColumns.toStructType, - StructType.fromAttributes(writeColumns), - SparkEnv.get.blockManager, - SparkEnv.get.serializerManager, - TaskContext.get().taskMemoryManager().pageSizeBytes, - SparkEnv.get.conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", - UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD)) - - while (iterator.hasNext) { - val currentRow = iterator.next() - sorter.insertKV(getSortingKey(currentRow), getOutputRow(currentRow)) - } - logDebug(s"Sorting complete. Writing out partition files one at a time.") - - val sortedIterator = sorter.sortedIterator() - val paths = new ArrayBuffer[Path] - - // Write the sorted data to partitioned files, one for each unique key - var currentWriter: OutputWriter = null - try { - var currentKey: UnsafeRow = null - while (sortedIterator.next()) { - val nextKey = sortedIterator.getKey - - // If key changes, close current writer, and open a new writer to a new partitioned file - if (currentKey != nextKey) { - if (currentWriter != null) { - currentWriter.close() - currentWriter = null - } - currentKey = nextKey.copy() - val partitionPath = getPartitionString(currentKey).getString(0) - val path = new Path(new Path(basePath, partitionPath), UUID.randomUUID.toString) - paths += path - currentWriter = newOutputWriter(path) - logInfo(s"Writing partition $currentKey to $path") - } - currentWriter.writeInternal(sortedIterator.getValue) - } - if (currentWriter != null) { - currentWriter.close() - currentWriter = null - } - if (paths.nonEmpty) { - val fs = paths.head.getFileSystem(serializableConf.value) - paths.map(p => SinkFileStatus(fs.getFileStatus(p))) - } else Seq.empty - } catch { - case cause: Throwable => - logError("Aborting task.", cause) - // call failure callbacks first, so we could have a chance to cleanup the writer. - TaskContext.get().asInstanceOf[TaskContextImpl].markTaskFailed(cause) - throw new SparkException("Task failed while writing rows.", cause) - } finally { - if (currentWriter != null) { - currentWriter.close() - } + FileFormatWriter.write( + sparkSession = sparkSession, + plan = data.logicalPlan, + fileFormat = fileFormat, + committer = committer, + outputPath = path, + hadoopConf = hadoopConf, + partitionColumns = partitionColumns, + bucketSpec = None, + refreshFunction = _ => (), + options = options) } } + + override def toString: String = s"FileSink[$path]" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala new file mode 100644 index 000000000000..510312267a98 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala @@ -0,0 +1,114 @@ +/* + * 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.streaming + +import java.util.UUID + +import scala.collection.mutable.ArrayBuffer + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.datasources.FileCommitProtocol +import org.apache.spark.sql.execution.datasources.FileCommitProtocol.TaskCommitMessage + +/** + * A [[FileCommitProtocol]] that tracks the list of valid files in a manifest file, used in + * structured streaming. + * + * @param path path to write the final output to. + */ +class ManifestFileCommitProtocol(path: String) + extends FileCommitProtocol with Serializable with Logging { + + // Track the list of files added by a task, only used on the executors. + @transient private var addedFiles: ArrayBuffer[String] = _ + + @transient private var fileLog: FileStreamSinkLog = _ + private var batchId: Long = _ + + /** + * Sets up the manifest log output and the batch id for this job. + * Must be called before any other function. + */ + def setupManifestOptions(fileLog: FileStreamSinkLog, batchId: Long): Unit = { + this.fileLog = fileLog + this.batchId = batchId + } + + override def setupJob(jobContext: JobContext): Unit = { + require(fileLog != null, "setupManifestOptions must be called before this function") + // Do nothing + } + + override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = { + require(fileLog != null, "setupManifestOptions must be called before this function") + val fileStatuses = taskCommits.flatMap(_.obj.asInstanceOf[Seq[SinkFileStatus]]).toArray + + if (fileLog.add(batchId, fileStatuses)) { + logInfo(s"Committed batch $batchId") + } else { + throw new IllegalStateException(s"Race while writing batch $batchId") + } + } + + override def abortJob(jobContext: JobContext): Unit = { + require(fileLog != null, "setupManifestOptions must be called before this function") + // Do nothing + } + + override def setupTask(taskContext: TaskAttemptContext): Unit = { + addedFiles = new ArrayBuffer[String] + } + + override def newTaskTempFile( + taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = { + // The file name looks like part-r-00000-2dd664f9-d2c4-4ffe-878f-c6c70c1fb0cb_00003.gz.parquet + // Note that %05d does not truncate the split number, so if we have more than 100000 tasks, + // the file name is fine and won't overflow. + val split = taskContext.getTaskAttemptID.getTaskID.getId + val uuid = UUID.randomUUID.toString + val filename = f"part-$split%05d-$uuid$ext" + + val file = dir.map { d => + new Path(new Path(path, d), filename).toString + }.getOrElse { + new Path(path, filename).toString + } + + addedFiles += file + file + } + + override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = { + if (addedFiles.nonEmpty) { + val fs = new Path(addedFiles.head).getFileSystem(taskContext.getConfiguration) + val statuses: Seq[SinkFileStatus] = + addedFiles.map(f => SinkFileStatus(fs.getFileStatus(new Path(f)))) + new TaskCommitMessage(statuses) + } else { + new TaskCommitMessage(Seq.empty[SinkFileStatus]) + } + } + + override def abortTask(taskContext: TaskAttemptContext): Unit = { + // Do nothing + // TODO: we can also try delete the addedFiles as a best-effort cleanup. + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 29e79847aa38..7bb3ac02fa5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -31,6 +31,7 @@ import org.apache.spark.internal.config._ import org.apache.spark.network.util.ByteUnit import org.apache.spark.sql.catalyst.CatalystConf import org.apache.spark.sql.execution.datasources.HadoopCommitProtocolWrapper +import org.apache.spark.sql.execution.streaming.ManifestFileCommitProtocol import org.apache.spark.util.Utils //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -523,7 +524,7 @@ object SQLConf { SQLConfigBuilder("spark.sql.streaming.commitProtocolClass") .internal() .stringConf - .createWithDefault(classOf[HadoopCommitProtocolWrapper].getName) + .createWithDefault(classOf[ManifestFileCommitProtocol].getName) val FILE_SINK_LOG_DELETION = SQLConfigBuilder("spark.sql.streaming.fileSink.log.deletion") .internal() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 18b42a81a098..902cf0534471 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -17,106 +17,16 @@ package org.apache.spark.sql.streaming -import java.io.File - -import org.apache.commons.io.FileUtils -import org.apache.commons.io.filefilter.{DirectoryFileFilter, RegexFileFilter} - import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.execution.streaming.{FileStreamSinkWriter, MemoryStream, MetadataLogFileIndex} -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.execution.streaming.{MemoryStream, MetadataLogFileIndex} import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.Utils class FileStreamSinkSuite extends StreamTest { import testImplicits._ - - test("FileStreamSinkWriter - unpartitioned data") { - val path = Utils.createTempDir() - path.delete() - - val hadoopConf = spark.sparkContext.hadoopConfiguration - val fileFormat = new parquet.ParquetFileFormat() - - def writeRange(start: Int, end: Int, numPartitions: Int): Seq[String] = { - val df = spark - .range(start, end, 1, numPartitions) - .select($"id", lit(100).as("data")) - val writer = new FileStreamSinkWriter( - df, fileFormat, path.toString, partitionColumnNames = Nil, hadoopConf, Map.empty) - writer.write().map(_.path.stripPrefix("file://")) - } - - // Write and check whether new files are written correctly - val files1 = writeRange(0, 10, 2) - assert(files1.size === 2, s"unexpected number of files: $files1") - checkFilesExist(path, files1, "file not written") - checkAnswer(spark.read.load(path.getCanonicalPath), (0 until 10).map(Row(_, 100))) - - // Append and check whether new files are written correctly and old files still exist - val files2 = writeRange(10, 20, 3) - assert(files2.size === 3, s"unexpected number of files: $files2") - assert(files2.intersect(files1).isEmpty, "old files returned") - checkFilesExist(path, files2, s"New file not written") - checkFilesExist(path, files1, s"Old file not found") - checkAnswer(spark.read.load(path.getCanonicalPath), (0 until 20).map(Row(_, 100))) - } - - test("FileStreamSinkWriter - partitioned data") { - implicit val e = ExpressionEncoder[java.lang.Long] - val path = Utils.createTempDir() - path.delete() - - val hadoopConf = spark.sparkContext.hadoopConfiguration - val fileFormat = new parquet.ParquetFileFormat() - - def writeRange(start: Int, end: Int, numPartitions: Int): Seq[String] = { - val df = spark - .range(start, end, 1, numPartitions) - .flatMap(x => Iterator(x, x, x)).toDF("id") - .select($"id", lit(100).as("data1"), lit(1000).as("data2")) - - require(df.rdd.partitions.size === numPartitions) - val writer = new FileStreamSinkWriter( - df, fileFormat, path.toString, partitionColumnNames = Seq("id"), hadoopConf, Map.empty) - writer.write().map(_.path.stripPrefix("file://")) - } - - def checkOneFileWrittenPerKey(keys: Seq[Int], filesWritten: Seq[String]): Unit = { - keys.foreach { id => - assert( - filesWritten.count(_.contains(s"/id=$id/")) == 1, - s"no file for id=$id. all files: \n\t${filesWritten.mkString("\n\t")}" - ) - } - } - - // Write and check whether new files are written correctly - val files1 = writeRange(0, 10, 2) - assert(files1.size === 10, s"unexpected number of files:\n${files1.mkString("\n")}") - checkFilesExist(path, files1, "file not written") - checkOneFileWrittenPerKey(0 until 10, files1) - - val answer1 = (0 until 10).flatMap(x => Iterator(x, x, x)).map(Row(100, 1000, _)) - checkAnswer(spark.read.load(path.getCanonicalPath), answer1) - - // Append and check whether new files are written correctly and old files still exist - val files2 = writeRange(0, 20, 3) - assert(files2.size === 20, s"unexpected number of files:\n${files2.mkString("\n")}") - assert(files2.intersect(files1).isEmpty, "old files returned") - checkFilesExist(path, files2, s"New file not written") - checkFilesExist(path, files1, s"Old file not found") - checkOneFileWrittenPerKey(0 until 20, files2) - - val answer2 = (0 until 20).flatMap(x => Iterator(x, x, x)).map(Row(100, 1000, _)) - checkAnswer(spark.read.load(path.getCanonicalPath), answer1 ++ answer2) - } - test("FileStreamSink - unpartitioned writing and batch reading") { val inputData = MemoryStream[Int] val df = inputData.toDF() @@ -270,18 +180,4 @@ class FileStreamSinkSuite extends StreamTest { } } - private def checkFilesExist(dir: File, expectedFiles: Seq[String], msg: String): Unit = { - import scala.collection.JavaConverters._ - val files = - FileUtils.listFiles(dir, new RegexFileFilter("[^.]+"), DirectoryFileFilter.DIRECTORY) - .asScala - .map(_.getCanonicalPath) - .toSet - - expectedFiles.foreach { f => - assert(files.contains(f), - s"\n$msg\nexpected file:\n\t$f\nfound files:\n${files.mkString("\n\t")}") - } - } - } From ad4832a9faf2c0c869bbcad9d71afe1cecbd3ec8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 1 Nov 2016 21:20:53 -0700 Subject: [PATCH 035/132] [SPARK-18216][SQL] Make Column.expr public ## What changes were proposed in this pull request? Column.expr is private[sql], but it's an actually really useful field to have for debugging. We should open it up, similar to how we use QueryExecution. ## How was this patch tested? N/A - this is a simple visibility change. Author: Reynold Xin Closes #15724 from rxin/SPARK-18216. --- sql/core/src/main/scala/org/apache/spark/sql/Column.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 05e867bf5be9..249408e0fbce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -118,6 +118,9 @@ class TypedColumn[-T, U]( * $"a" === $"b" * }}} * + * Note that the internal Catalyst expression can be accessed via "expr", but this method is for + * debugging purposes only and can change in any future Spark releases. + * * @groupname java_expr_ops Java-specific expression operators * @groupname expr_ops Expression operators * @groupname df_ops DataFrame functions @@ -126,7 +129,7 @@ class TypedColumn[-T, U]( * @since 1.3.0 */ @InterfaceStability.Stable -class Column(protected[sql] val expr: Expression) extends Logging { +class Column(val expr: Expression) extends Logging { def this(name: String) = this(name match { case "*" => UnresolvedStar(None) From 1ecfafa0869cb3a3e367bda8be252a69874dc4de Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Tue, 1 Nov 2016 22:14:53 -0700 Subject: [PATCH 036/132] [SPARK-17838][SPARKR] Check named arguments for options and use formatted R friendly message from JVM exception message ## What changes were proposed in this pull request? This PR proposes to - improve the R-friendly error messages rather than raw JVM exception one. As `read.json`, `read.text`, `read.orc`, `read.parquet` and `read.jdbc` are executed in the same path with `read.df`, and `write.json`, `write.text`, `write.orc`, `write.parquet` and `write.jdbc` shares the same path with `write.df`, it seems it is safe to call `handledCallJMethod` to handle JVM messages. - prevent `zero-length variable name` and prints the ignored options as an warning message. **Before** ``` r > read.json("path", a = 1, 2, 3, "a") Error in env[[name]] <- value : zero-length variable name ``` ``` r > read.json("arbitrary_path") Error in invokeJava(isStatic = FALSE, objId$id, methodName, ...) : org.apache.spark.sql.AnalysisException: Path does not exist: file:/...; at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$12.apply(DataSource.scala:398) ... > read.orc("arbitrary_path") Error in invokeJava(isStatic = FALSE, objId$id, methodName, ...) : org.apache.spark.sql.AnalysisException: Path does not exist: file:/...; at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$12.apply(DataSource.scala:398) ... > read.text("arbitrary_path") Error in invokeJava(isStatic = FALSE, objId$id, methodName, ...) : org.apache.spark.sql.AnalysisException: Path does not exist: file:/...; at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$12.apply(DataSource.scala:398) ... > read.parquet("arbitrary_path") Error in invokeJava(isStatic = FALSE, objId$id, methodName, ...) : org.apache.spark.sql.AnalysisException: Path does not exist: file:/...; at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$12.apply(DataSource.scala:398) ... ``` ``` r > write.json(df, "existing_path") Error in invokeJava(isStatic = FALSE, objId$id, methodName, ...) : org.apache.spark.sql.AnalysisException: path file:/... already exists.; at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:68) > write.orc(df, "existing_path") Error in invokeJava(isStatic = FALSE, objId$id, methodName, ...) : org.apache.spark.sql.AnalysisException: path file:/... already exists.; at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:68) > write.text(df, "existing_path") Error in invokeJava(isStatic = FALSE, objId$id, methodName, ...) : org.apache.spark.sql.AnalysisException: path file:/... already exists.; at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:68) > write.parquet(df, "existing_path") Error in invokeJava(isStatic = FALSE, objId$id, methodName, ...) : org.apache.spark.sql.AnalysisException: path file:/... already exists.; at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:68) ``` **After** ``` r read.json("arbitrary_path", a = 1, 2, 3, "a") Unnamed arguments ignored: 2, 3, a. ``` ``` r > read.json("arbitrary_path") Error in json : analysis error - Path does not exist: file:/... > read.orc("arbitrary_path") Error in orc : analysis error - Path does not exist: file:/... > read.text("arbitrary_path") Error in text : analysis error - Path does not exist: file:/... > read.parquet("arbitrary_path") Error in parquet : analysis error - Path does not exist: file:/... ``` ``` r > write.json(df, "existing_path") Error in json : analysis error - path file:/... already exists.; > write.orc(df, "existing_path") Error in orc : analysis error - path file:/... already exists.; > write.text(df, "existing_path") Error in text : analysis error - path file:/... already exists.; > write.parquet(df, "existing_path") Error in parquet : analysis error - path file:/... already exists.; ``` ## How was this patch tested? Unit tests in `test_utils.R` and `test_sparkSQL.R`. Author: hyukjinkwon Closes #15608 from HyukjinKwon/SPARK-17838. --- R/pkg/R/DataFrame.R | 10 +++--- R/pkg/R/SQLContext.R | 17 ++++----- R/pkg/R/utils.R | 44 ++++++++++++++++------- R/pkg/inst/tests/testthat/test_sparkSQL.R | 16 +++++++++ R/pkg/inst/tests/testthat/test_utils.R | 2 ++ 5 files changed, 64 insertions(+), 25 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 1df8bbf9fe60..1cf9b38ea648 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -788,7 +788,7 @@ setMethod("write.json", function(x, path, mode = "error", ...) { write <- callJMethod(x@sdf, "write") write <- setWriteOptions(write, mode = mode, ...) - invisible(callJMethod(write, "json", path)) + invisible(handledCallJMethod(write, "json", path)) }) #' Save the contents of SparkDataFrame as an ORC file, preserving the schema. @@ -819,7 +819,7 @@ setMethod("write.orc", function(x, path, mode = "error", ...) { write <- callJMethod(x@sdf, "write") write <- setWriteOptions(write, mode = mode, ...) - invisible(callJMethod(write, "orc", path)) + invisible(handledCallJMethod(write, "orc", path)) }) #' Save the contents of SparkDataFrame as a Parquet file, preserving the schema. @@ -851,7 +851,7 @@ setMethod("write.parquet", function(x, path, mode = "error", ...) { write <- callJMethod(x@sdf, "write") write <- setWriteOptions(write, mode = mode, ...) - invisible(callJMethod(write, "parquet", path)) + invisible(handledCallJMethod(write, "parquet", path)) }) #' @rdname write.parquet @@ -895,7 +895,7 @@ setMethod("write.text", function(x, path, mode = "error", ...) { write <- callJMethod(x@sdf, "write") write <- setWriteOptions(write, mode = mode, ...) - invisible(callJMethod(write, "text", path)) + invisible(handledCallJMethod(write, "text", path)) }) #' Distinct @@ -3342,7 +3342,7 @@ setMethod("write.jdbc", jprops <- varargsToJProperties(...) write <- callJMethod(x@sdf, "write") write <- callJMethod(write, "mode", jmode) - invisible(callJMethod(write, "jdbc", url, tableName, jprops)) + invisible(handledCallJMethod(write, "jdbc", url, tableName, jprops)) }) #' randomSplit diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 216ca51666ba..38d83c6e5c52 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -350,7 +350,7 @@ read.json.default <- function(path, ...) { paths <- as.list(suppressWarnings(normalizePath(path))) read <- callJMethod(sparkSession, "read") read <- callJMethod(read, "options", options) - sdf <- callJMethod(read, "json", paths) + sdf <- handledCallJMethod(read, "json", paths) dataFrame(sdf) } @@ -422,7 +422,7 @@ read.orc <- function(path, ...) { path <- suppressWarnings(normalizePath(path)) read <- callJMethod(sparkSession, "read") read <- callJMethod(read, "options", options) - sdf <- callJMethod(read, "orc", path) + sdf <- handledCallJMethod(read, "orc", path) dataFrame(sdf) } @@ -444,7 +444,7 @@ read.parquet.default <- function(path, ...) { paths <- as.list(suppressWarnings(normalizePath(path))) read <- callJMethod(sparkSession, "read") read <- callJMethod(read, "options", options) - sdf <- callJMethod(read, "parquet", paths) + sdf <- handledCallJMethod(read, "parquet", paths) dataFrame(sdf) } @@ -496,7 +496,7 @@ read.text.default <- function(path, ...) { paths <- as.list(suppressWarnings(normalizePath(path))) read <- callJMethod(sparkSession, "read") read <- callJMethod(read, "options", options) - sdf <- callJMethod(read, "text", paths) + sdf <- handledCallJMethod(read, "text", paths) dataFrame(sdf) } @@ -914,12 +914,13 @@ read.jdbc <- function(url, tableName, } else { numPartitions <- numToInt(numPartitions) } - sdf <- callJMethod(read, "jdbc", url, tableName, as.character(partitionColumn), - numToInt(lowerBound), numToInt(upperBound), numPartitions, jprops) + sdf <- handledCallJMethod(read, "jdbc", url, tableName, as.character(partitionColumn), + numToInt(lowerBound), numToInt(upperBound), numPartitions, jprops) } else if (length(predicates) > 0) { - sdf <- callJMethod(read, "jdbc", url, tableName, as.list(as.character(predicates)), jprops) + sdf <- handledCallJMethod(read, "jdbc", url, tableName, as.list(as.character(predicates)), + jprops) } else { - sdf <- callJMethod(read, "jdbc", url, tableName, jprops) + sdf <- handledCallJMethod(read, "jdbc", url, tableName, jprops) } dataFrame(sdf) } diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index c4e78cbb804d..20004549cc03 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -338,21 +338,41 @@ varargsToEnv <- function(...) { # into string. varargsToStrEnv <- function(...) { pairs <- list(...) + nameList <- names(pairs) env <- new.env() - for (name in names(pairs)) { - value <- pairs[[name]] - if (!(is.logical(value) || is.numeric(value) || is.character(value) || is.null(value))) { - stop(paste0("Unsupported type for ", name, " : ", class(value), - ". Supported types are logical, numeric, character and NULL.")) - } - if (is.logical(value)) { - env[[name]] <- tolower(as.character(value)) - } else if (is.null(value)) { - env[[name]] <- value - } else { - env[[name]] <- as.character(value) + ignoredNames <- list() + + if (is.null(nameList)) { + # When all arguments are not named, names(..) returns NULL. + ignoredNames <- pairs + } else { + for (i in seq_along(pairs)) { + name <- nameList[i] + value <- pairs[i] + if (identical(name, "")) { + # When some of arguments are not named, name is "". + ignoredNames <- append(ignoredNames, value) + } else { + value <- pairs[[name]] + if (!(is.logical(value) || is.numeric(value) || is.character(value) || is.null(value))) { + stop(paste0("Unsupported type for ", name, " : ", class(value), + ". Supported types are logical, numeric, character and NULL."), call. = FALSE) + } + if (is.logical(value)) { + env[[name]] <- tolower(as.character(value)) + } else if (is.null(value)) { + env[[name]] <- value + } else { + env[[name]] <- as.character(value) + } + } } } + + if (length(ignoredNames) != 0) { + warning(paste0("Unnamed arguments ignored: ", paste(ignoredNames, collapse = ", "), "."), + call. = FALSE) + } env } diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 9289db57b6d6..806019d7524f 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -2660,6 +2660,14 @@ test_that("Call DataFrameWriter.save() API in Java without path and check argume # DataFrameWriter.save() without path. expect_error(write.df(df, source = "csv"), "Error in save : illegal argument - 'path' is not specified") + expect_error(write.json(df, jsonPath), + "Error in json : analysis error - path file:.*already exists") + expect_error(write.text(df, jsonPath), + "Error in text : analysis error - path file:.*already exists") + expect_error(write.orc(df, jsonPath), + "Error in orc : analysis error - path file:.*already exists") + expect_error(write.parquet(df, jsonPath), + "Error in parquet : analysis error - path file:.*already exists") # Arguments checking in R side. expect_error(write.df(df, "data.tmp", source = c(1, 2)), @@ -2679,6 +2687,11 @@ test_that("Call DataFrameWriter.load() API in Java without path and check argume paste("Error in loadDF : analysis error - Unable to infer schema for JSON at .", "It must be specified manually")) expect_error(read.df("arbitrary_path"), "Error in loadDF : analysis error - Path does not exist") + expect_error(read.json("arbitrary_path"), "Error in json : analysis error - Path does not exist") + expect_error(read.text("arbitrary_path"), "Error in text : analysis error - Path does not exist") + expect_error(read.orc("arbitrary_path"), "Error in orc : analysis error - Path does not exist") + expect_error(read.parquet("arbitrary_path"), + "Error in parquet : analysis error - Path does not exist") # Arguments checking in R side. expect_error(read.df(path = c(3)), @@ -2686,6 +2699,9 @@ test_that("Call DataFrameWriter.load() API in Java without path and check argume expect_error(read.df(jsonPath, source = c(1, 2)), paste("source should be character, NULL or omitted. It is the datasource specified", "in 'spark.sql.sources.default' configuration by default.")) + + expect_warning(read.json(jsonPath, a = 1, 2, 3, "a"), + "Unnamed arguments ignored: 2, 3, a.") }) unlink(parquetPath) diff --git a/R/pkg/inst/tests/testthat/test_utils.R b/R/pkg/inst/tests/testthat/test_utils.R index a20254e9b3fa..607c407f04f9 100644 --- a/R/pkg/inst/tests/testthat/test_utils.R +++ b/R/pkg/inst/tests/testthat/test_utils.R @@ -224,6 +224,8 @@ test_that("varargsToStrEnv", { expect_error(varargsToStrEnv(a = list(1, "a")), paste0("Unsupported type for a : list. Supported types are logical, ", "numeric, character and NULL.")) + expect_warning(varargsToStrEnv(a = 1, 2, 3, 4), "Unnamed arguments ignored: 2, 3, 4.") + expect_warning(varargsToStrEnv(1, 2, 3, 4), "Unnamed arguments ignored: 1, 2, 3, 4.") }) sparkR.session.stop() From 1bbf9ff634745148e782370009aa31d3a042638c Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Tue, 1 Nov 2016 22:20:19 -0700 Subject: [PATCH 037/132] [SPARK-17992][SQL] Return all partitions from HiveShim when Hive throws a metastore exception when attempting to fetch partitions by filter (Link to Jira issue: https://issues.apache.org/jira/browse/SPARK-17992) ## What changes were proposed in this pull request? We recently added table partition pruning for partitioned Hive tables converted to using `TableFileCatalog`. When the Hive configuration option `hive.metastore.try.direct.sql` is set to `false`, Hive will throw an exception for unsupported filter expressions. For example, attempting to filter on an integer partition column will throw a `org.apache.hadoop.hive.metastore.api.MetaException`. I discovered this behavior because VideoAmp uses the CDH version of Hive with a Postgresql metastore DB. In this configuration, CDH sets `hive.metastore.try.direct.sql` to `false` by default, and queries that filter on a non-string partition column will fail. Rather than throw an exception in query planning, this patch catches this exception, logs a warning and returns all table partitions instead. Clients of this method are already expected to handle the possibility that the filters will not be honored. ## How was this patch tested? A unit test was added. Author: Michael Allman Closes #15673 from mallman/spark-17992-catch_hive_partition_filter_exception. --- .../spark/sql/hive/client/HiveShim.scala | 31 ++++++-- .../sql/hive/client/HiveClientBuilder.scala | 56 ++++++++++++++ .../sql/hive/client/HiveClientSuite.scala | 61 +++++++++++++++ .../spark/sql/hive/client/VersionsSuite.scala | 77 +++++-------------- 4 files changed, 160 insertions(+), 65 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientBuilder.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index 85edaf63db88..3d9642dd1463 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -29,7 +29,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.api.{Function => HiveFunction, FunctionType, NoSuchObjectException, PrincipalType, ResourceType, ResourceUri} +import org.apache.hadoop.hive.metastore.api.{Function => HiveFunction, FunctionType, MetaException, PrincipalType, ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition, Table} import org.apache.hadoop.hive.ql.plan.AddPartitionDesc @@ -43,6 +43,7 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTablePartition, FunctionResource, FunctionResourceType} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegralType, StringType} import org.apache.spark.util.Utils @@ -586,17 +587,31 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]] } else { logDebug(s"Hive metastore filter is '$filter'.") + val tryDirectSqlConfVar = HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL + val tryDirectSql = + hive.getConf.getBoolean(tryDirectSqlConfVar.varname, tryDirectSqlConfVar.defaultBoolVal) try { + // Hive may throw an exception when calling this method in some circumstances, such as + // when filtering on a non-string partition column when the hive config key + // hive.metastore.try.direct.sql is false getPartitionsByFilterMethod.invoke(hive, table, filter) .asInstanceOf[JArrayList[Partition]] } catch { - case e: InvocationTargetException => - // SPARK-18167 retry to investigate the flaky test. This should be reverted before - // the release is cut. - val retry = Try(getPartitionsByFilterMethod.invoke(hive, table, filter)) - logError("getPartitionsByFilter failed, retry success = " + retry.isSuccess) - logError("all partitions: " + getAllPartitions(hive, table)) - throw e + case ex: InvocationTargetException if ex.getCause.isInstanceOf[MetaException] && + !tryDirectSql => + logWarning("Caught Hive MetaException attempting to get partition metadata by " + + "filter from Hive. Falling back to fetching all partition metadata, which will " + + "degrade performance. Modifying your Hive metastore configuration to set " + + s"${tryDirectSqlConfVar.varname} to true may resolve this problem.", ex) + // HiveShim clients are expected to handle a superset of the requested partitions + getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]] + case ex: InvocationTargetException if ex.getCause.isInstanceOf[MetaException] && + tryDirectSql => + throw new RuntimeException("Caught Hive MetaException attempting to get partition " + + "metadata by filter from Hive. You can set the Spark configuration setting " + + s"${SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key} to false to work around this " + + "problem, however this will result in degraded performance. Please report a bug: " + + "https://issues.apache.org/jira/browse/SPARK", ex) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientBuilder.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientBuilder.scala new file mode 100644 index 000000000000..591a968c8284 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientBuilder.scala @@ -0,0 +1,56 @@ +/* + * 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.client + +import java.io.File + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.util.VersionInfo + +import org.apache.spark.SparkConf +import org.apache.spark.util.Utils + +private[client] class HiveClientBuilder { + private val sparkConf = new SparkConf() + + // In order to speed up test execution during development or in Jenkins, you can specify the path + // of an existing Ivy cache: + private val ivyPath: Option[String] = { + sys.env.get("SPARK_VERSIONS_SUITE_IVY_PATH").orElse( + Some(new File(sys.props("java.io.tmpdir"), "hive-ivy-cache").getAbsolutePath)) + } + + private def buildConf() = { + lazy val warehousePath = Utils.createTempDir() + lazy val metastorePath = Utils.createTempDir() + metastorePath.delete() + Map( + "javax.jdo.option.ConnectionURL" -> s"jdbc:derby:;databaseName=$metastorePath;create=true", + "hive.metastore.warehouse.dir" -> warehousePath.toString) + } + + def buildClient(version: String, hadoopConf: Configuration): HiveClient = { + IsolatedClientLoader.forVersion( + hiveMetastoreVersion = version, + hadoopVersion = VersionInfo.getVersion, + sparkConf = sparkConf, + hadoopConf = hadoopConf, + config = buildConf(), + ivyPath = ivyPath).createClient() + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala new file mode 100644 index 000000000000..4790331168bd --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala @@ -0,0 +1,61 @@ +/* + * 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.client + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.conf.HiveConf + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} +import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.types.IntegerType + +class HiveClientSuite extends SparkFunSuite { + private val clientBuilder = new HiveClientBuilder + + private val tryDirectSqlKey = HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname + + test(s"getPartitionsByFilter returns all partitions when $tryDirectSqlKey=false") { + val testPartitionCount = 5 + + val storageFormat = CatalogStorageFormat( + locationUri = None, + inputFormat = None, + outputFormat = None, + serde = None, + compressed = false, + properties = Map.empty) + + val hadoopConf = new Configuration() + hadoopConf.setBoolean(tryDirectSqlKey, false) + val client = clientBuilder.buildClient(HiveUtils.hiveExecutionVersion, hadoopConf) + client.runSqlHive("CREATE TABLE test (value INT) PARTITIONED BY (part INT)") + + val partitions = (1 to testPartitionCount).map { part => + CatalogTablePartition(Map("part" -> part.toString), storageFormat) + } + client.createPartitions( + "default", "test", partitions, ignoreIfExists = false) + + val filteredPartitions = client.getPartitionsByFilter(client.getTable("default", "test"), + Seq(EqualTo(AttributeReference("part", IntegerType)(), Literal(3)))) + + assert(filteredPartitions.size == testPartitionCount) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 9a10957c8efa..081b0ed9bd68 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -23,9 +23,8 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.mapred.TextInputFormat -import org.apache.hadoop.util.VersionInfo -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} @@ -48,46 +47,19 @@ import org.apache.spark.util.{MutableURLClassLoader, Utils} @ExtendedHiveTest class VersionsSuite extends SparkFunSuite with Logging { - private val sparkConf = new SparkConf() - - // In order to speed up test execution during development or in Jenkins, you can specify the path - // of an existing Ivy cache: - private val ivyPath: Option[String] = { - sys.env.get("SPARK_VERSIONS_SUITE_IVY_PATH").orElse( - Some(new File(sys.props("java.io.tmpdir"), "hive-ivy-cache").getAbsolutePath)) - } - - private def buildConf() = { - lazy val warehousePath = Utils.createTempDir() - lazy val metastorePath = Utils.createTempDir() - metastorePath.delete() - Map( - "javax.jdo.option.ConnectionURL" -> s"jdbc:derby:;databaseName=$metastorePath;create=true", - "hive.metastore.warehouse.dir" -> warehousePath.toString) - } + private val clientBuilder = new HiveClientBuilder + import clientBuilder.buildClient test("success sanity check") { - val badClient = IsolatedClientLoader.forVersion( - hiveMetastoreVersion = HiveUtils.hiveExecutionVersion, - hadoopVersion = VersionInfo.getVersion, - sparkConf = sparkConf, - hadoopConf = new Configuration(), - config = buildConf(), - ivyPath = ivyPath).createClient() + val badClient = buildClient(HiveUtils.hiveExecutionVersion, new Configuration()) val db = new CatalogDatabase("default", "desc", "loc", Map()) badClient.createDatabase(db, ignoreIfExists = true) } test("hadoop configuration preserved") { - val hadoopConf = new Configuration(); + val hadoopConf = new Configuration() hadoopConf.set("test", "success") - val client = IsolatedClientLoader.forVersion( - hiveMetastoreVersion = HiveUtils.hiveExecutionVersion, - hadoopVersion = VersionInfo.getVersion, - sparkConf = sparkConf, - hadoopConf = hadoopConf, - config = buildConf(), - ivyPath = ivyPath).createClient() + val client = buildClient(HiveUtils.hiveExecutionVersion, hadoopConf) assert("success" === client.getConf("test", null)) } @@ -109,15 +81,7 @@ class VersionsSuite extends SparkFunSuite with Logging { // TODO: currently only works on mysql where we manually create the schema... ignore("failure sanity check") { val e = intercept[Throwable] { - val badClient = quietly { - IsolatedClientLoader.forVersion( - hiveMetastoreVersion = "13", - hadoopVersion = VersionInfo.getVersion, - sparkConf = sparkConf, - hadoopConf = new Configuration(), - config = buildConf(), - ivyPath = ivyPath).createClient() - } + val badClient = quietly { buildClient("13", new Configuration()) } } assert(getNestedMessages(e) contains "Unknown column 'A0.OWNER_NAME' in 'field list'") } @@ -130,16 +94,9 @@ class VersionsSuite extends SparkFunSuite with Logging { test(s"$version: create client") { client = null System.gc() // Hack to avoid SEGV on some JVM versions. - val hadoopConf = new Configuration(); + val hadoopConf = new Configuration() hadoopConf.set("test", "success") - client = - IsolatedClientLoader.forVersion( - hiveMetastoreVersion = version, - hadoopVersion = VersionInfo.getVersion, - sparkConf = sparkConf, - hadoopConf = hadoopConf, - config = buildConf(), - ivyPath = ivyPath).createClient() + client = buildClient(version, hadoopConf) } def table(database: String, tableName: String): CatalogTable = { @@ -287,15 +244,19 @@ class VersionsSuite extends SparkFunSuite with Logging { client.runSqlHive("CREATE TABLE src_part (value INT) PARTITIONED BY (key1 INT, key2 INT)") } + val testPartitionCount = 2 + test(s"$version: createPartitions") { - val partition1 = CatalogTablePartition(Map("key1" -> "1", "key2" -> "1"), storageFormat) - val partition2 = CatalogTablePartition(Map("key1" -> "1", "key2" -> "2"), storageFormat) + val partitions = (1 to testPartitionCount).map { key2 => + CatalogTablePartition(Map("key1" -> "1", "key2" -> key2.toString), storageFormat) + } client.createPartitions( - "default", "src_part", Seq(partition1, partition2), ignoreIfExists = true) + "default", "src_part", partitions, ignoreIfExists = true) } test(s"$version: getPartitions(catalogTable)") { - assert(2 == client.getPartitions(client.getTable("default", "src_part")).size) + assert(testPartitionCount == + client.getPartitions(client.getTable("default", "src_part")).size) } test(s"$version: getPartitionsByFilter") { @@ -306,6 +267,8 @@ class VersionsSuite extends SparkFunSuite with Logging { // Hive 0.12 doesn't support getPartitionsByFilter, it ignores the filter condition. if (version != "0.12") { assert(result.size == 1) + } else { + assert(result.size == testPartitionCount) } } @@ -327,7 +290,7 @@ class VersionsSuite extends SparkFunSuite with Logging { } test(s"$version: getPartitions(db: String, table: String)") { - assert(2 == client.getPartitions("default", "src_part", None).size) + assert(testPartitionCount == client.getPartitions("default", "src_part", None).size) } test(s"$version: loadPartition") { From 620da3b4828b3580c7ed7339b2a07938e6be1bb1 Mon Sep 17 00:00:00 2001 From: frreiss Date: Tue, 1 Nov 2016 23:00:17 -0700 Subject: [PATCH 038/132] [SPARK-17475][STREAMING] Delete CRC files if the filesystem doesn't use checksum files ## What changes were proposed in this pull request? When the metadata logs for various parts of Structured Streaming are stored on non-HDFS filesystems such as NFS or ext4, the HDFSMetadataLog class leaves hidden HDFS-style checksum (CRC) files in the log directory, one file per batch. This PR modifies HDFSMetadataLog so that it detects the use of a filesystem that doesn't use CRC files and removes the CRC files. ## How was this patch tested? Modified an existing test case in HDFSMetadataLogSuite to check whether HDFSMetadataLog correctly removes CRC files on the local POSIX filesystem. Ran the entire regression suite. Author: frreiss Closes #15027 from frreiss/fred-17475. --- .../spark/sql/execution/streaming/HDFSMetadataLog.scala | 5 +++++ .../sql/execution/streaming/HDFSMetadataLogSuite.scala | 6 ++++++ 2 files changed, 11 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index c7235320fd6b..9a0f87cf0498 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -148,6 +148,11 @@ class HDFSMetadataLog[T: ClassTag](sparkSession: SparkSession, path: String) // It will fail if there is an existing file (someone has committed the batch) logDebug(s"Attempting to write log #${batchIdToPath(batchId)}") fileManager.rename(tempPath, batchIdToPath(batchId)) + + // SPARK-17475: HDFSMetadataLog should not leak CRC files + // If the underlying filesystem didn't rename the CRC file, delete it. + val crcPath = new Path(tempPath.getParent(), s".${tempPath.getName()}.crc") + if (fileManager.exists(crcPath)) fileManager.delete(crcPath) return } catch { case e: IOException if isFileAlreadyExistsException(e) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala index 9c1d26dcb224..d03e08d9a576 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala @@ -119,6 +119,12 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { assert(metadataLog.get(1).isEmpty) assert(metadataLog.get(2).isDefined) assert(metadataLog.getLatest().get._1 == 2) + + // There should be exactly one file, called "2", in the metadata directory. + // This check also tests for regressions of SPARK-17475 + val allFiles = new File(metadataLog.metadataPath.toString).listFiles().toSeq + assert(allFiles.size == 1) + assert(allFiles(0).getName() == "2") } } From abefe2ec428dc24a4112c623fb6fbe4b2ca60a2b Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 2 Nov 2016 14:15:10 +0800 Subject: [PATCH 039/132] [SPARK-18183][SPARK-18184] Fix INSERT [INTO|OVERWRITE] TABLE ... PARTITION for Datasource tables ## What changes were proposed in this pull request? There are a couple issues with the current 2.1 behavior when inserting into Datasource tables with partitions managed by Hive. (1) OVERWRITE TABLE ... PARTITION will actually overwrite the entire table instead of just the specified partition. (2) INSERT|OVERWRITE does not work with partitions that have custom locations. This PR fixes both of these issues for Datasource tables managed by Hive. The behavior for legacy tables or when `manageFilesourcePartitions = false` is unchanged. There is one other issue in that INSERT OVERWRITE with dynamic partitions will overwrite the entire table instead of just the updated partitions, but this behavior is pretty complicated to implement for Datasource tables. We should address that in a future release. ## How was this patch tested? Unit tests. Author: Eric Liang Closes #15705 from ericl/sc-4942. --- .../spark/sql/catalyst/dsl/package.scala | 2 +- .../sql/catalyst/parser/AstBuilder.scala | 9 +++- .../plans/logical/basicLogicalOperators.scala | 19 ++++++- .../sql/catalyst/parser/PlanParserSuite.scala | 15 ++++-- .../apache/spark/sql/DataFrameWriter.scala | 4 +- .../datasources/CatalogFileIndex.scala | 5 +- .../datasources/DataSourceStrategy.scala | 30 +++++++++-- .../InsertIntoDataSourceCommand.scala | 6 +-- .../spark/sql/hive/HiveStrategies.scala | 3 +- .../CreateHiveTableAsSelectCommand.scala | 5 +- .../PartitionProviderCompatibilitySuite.scala | 52 +++++++++++++++++++ 11 files changed, 129 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 66e52ca68af1..e901683be685 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -367,7 +367,7 @@ package object dsl { def insertInto(tableName: String, overwrite: Boolean = false): LogicalPlan = InsertIntoTable( analysis.UnresolvedRelation(TableIdentifier(tableName)), - Map.empty, logicalPlan, overwrite, false) + Map.empty, logicalPlan, OverwriteOptions(overwrite), false) def as(alias: String): LogicalPlan = logicalPlan match { case UnresolvedRelation(tbl, _) => UnresolvedRelation(tbl, Option(alias)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 38e9bb6c162a..ac1577b3abb4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -177,12 +177,19 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { throw new ParseException(s"Dynamic partitions do not support IF NOT EXISTS. Specified " + "partitions with value: " + dynamicPartitionKeys.keys.mkString("[", ",", "]"), ctx) } + val overwrite = ctx.OVERWRITE != null + val overwritePartition = + if (overwrite && partitionKeys.nonEmpty && dynamicPartitionKeys.isEmpty) { + Some(partitionKeys.map(t => (t._1, t._2.get))) + } else { + None + } InsertIntoTable( UnresolvedRelation(tableIdent, None), partitionKeys, query, - ctx.OVERWRITE != null, + OverwriteOptions(overwrite, overwritePartition), ctx.EXISTS != null) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index a48974c6322a..7a15c2285d58 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation +import org.apache.spark.sql.catalyst.catalog.CatalogTypes import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ @@ -345,18 +346,32 @@ case class BroadcastHint(child: LogicalPlan) extends UnaryNode { override lazy val statistics: Statistics = super.statistics.copy(isBroadcastable = true) } +/** + * Options for writing new data into a table. + * + * @param enabled whether to overwrite existing data in the table. + * @param specificPartition only data in the specified partition will be overwritten. + */ +case class OverwriteOptions( + enabled: Boolean, + specificPartition: Option[CatalogTypes.TablePartitionSpec] = None) { + if (specificPartition.isDefined) { + assert(enabled, "Overwrite must be enabled when specifying a partition to overwrite.") + } +} + case class InsertIntoTable( table: LogicalPlan, partition: Map[String, Option[String]], child: LogicalPlan, - overwrite: Boolean, + overwrite: OverwriteOptions, ifNotExists: Boolean) extends LogicalPlan { override def children: Seq[LogicalPlan] = child :: Nil override def output: Seq[Attribute] = Seq.empty - assert(overwrite || !ifNotExists) + assert(overwrite.enabled || !ifNotExists) assert(partition.values.forall(_.nonEmpty) || !ifNotExists) override lazy val resolved: Boolean = childrenResolved && table.resolved diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index ca86304d4d40..7400f3430e99 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -180,7 +180,16 @@ class PlanParserSuite extends PlanTest { partition: Map[String, Option[String]], overwrite: Boolean = false, ifNotExists: Boolean = false): LogicalPlan = - InsertIntoTable(table("s"), partition, plan, overwrite, ifNotExists) + InsertIntoTable( + table("s"), partition, plan, + OverwriteOptions( + overwrite, + if (overwrite && partition.nonEmpty) { + Some(partition.map(kv => (kv._1, kv._2.get))) + } else { + None + }), + ifNotExists) // Single inserts assertEqual(s"insert overwrite table s $sql", @@ -196,9 +205,9 @@ class PlanParserSuite extends PlanTest { val plan2 = table("t").where('x > 5).select(star()) assertEqual("from t insert into s select * limit 1 insert into u select * where x > 5", InsertIntoTable( - table("s"), Map.empty, plan.limit(1), overwrite = false, ifNotExists = false).union( + table("s"), Map.empty, plan.limit(1), OverwriteOptions(false), ifNotExists = false).union( InsertIntoTable( - table("u"), Map.empty, plan2, overwrite = false, ifNotExists = false))) + table("u"), Map.empty, plan2, OverwriteOptions(false), ifNotExists = false))) } test ("insert with if not exists") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 11dd1df90993..700f4835ac89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -25,7 +25,7 @@ import org.apache.spark.annotation.InterfaceStability import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType} -import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, Union} +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, OverwriteOptions, Union} import org.apache.spark.sql.execution.command.AlterTableRecoverPartitionsCommand import org.apache.spark.sql.execution.datasources.{CaseInsensitiveMap, CreateTable, DataSource, HadoopFsRelation} import org.apache.spark.sql.types.StructType @@ -259,7 +259,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { table = UnresolvedRelation(tableIdent), partition = Map.empty[String, Option[String]], child = df.logicalPlan, - overwrite = mode == SaveMode.Overwrite, + overwrite = OverwriteOptions(mode == SaveMode.Overwrite), ifNotExists = false)).toRdd } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala index 092aabc89a36..443a2ec033a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala @@ -67,7 +67,10 @@ class CatalogFileIndex( val selectedPartitions = sparkSession.sessionState.catalog.listPartitionsByFilter( table.identifier, filters) val partitions = selectedPartitions.map { p => - PartitionPath(p.toRow(partitionSchema), p.storage.locationUri.get) + val path = new Path(p.storage.locationUri.get) + val fs = path.getFileSystem(hadoopConf) + PartitionPath( + p.toRow(partitionSchema), path.makeQualified(fs.getUri, fs.getWorkingDirectory)) } val partitionSpec = PartitionSpec(partitionSchema, partitions) new PrunedInMemoryFileIndex( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 34b77cab65de..47c1f9d3fac1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.datasources import scala.collection.mutable.ArrayBuffer +import org.apache.hadoop.fs.Path + import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ @@ -174,14 +176,32 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { case LogicalRelation(r: HadoopFsRelation, _, _) => r.location.rootPaths }.flatten - val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append - if (overwrite && inputPaths.contains(outputPath)) { + val mode = if (overwrite.enabled) SaveMode.Overwrite else SaveMode.Append + if (overwrite.enabled && inputPaths.contains(outputPath)) { throw new AnalysisException( "Cannot overwrite a path that is also being read from.") } + val overwritingSinglePartition = (overwrite.specificPartition.isDefined && + t.sparkSession.sessionState.conf.manageFilesourcePartitions && + l.catalogTable.get.partitionProviderIsHive) + + val effectiveOutputPath = if (overwritingSinglePartition) { + val partition = t.sparkSession.sessionState.catalog.getPartition( + l.catalogTable.get.identifier, overwrite.specificPartition.get) + new Path(partition.storage.locationUri.get) + } else { + outputPath + } + + val effectivePartitionSchema = if (overwritingSinglePartition) { + Nil + } else { + query.resolve(t.partitionSchema, t.sparkSession.sessionState.analyzer.resolver) + } + def refreshPartitionsCallback(updatedPartitions: Seq[TablePartitionSpec]): Unit = { - if (l.catalogTable.isDefined && + if (l.catalogTable.isDefined && updatedPartitions.nonEmpty && l.catalogTable.get.partitionColumnNames.nonEmpty && l.catalogTable.get.partitionProviderIsHive) { val metastoreUpdater = AlterTableAddPartitionCommand( @@ -194,8 +214,8 @@ case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] { } val insertCmd = InsertIntoHadoopFsRelationCommand( - outputPath, - query.resolve(t.partitionSchema, t.sparkSession.sessionState.analyzer.resolver), + effectiveOutputPath, + effectivePartitionSchema, t.bucketSpec, t.fileFormat, refreshPartitionsCallback, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala index b2ff68a833fe..2eba1e9986ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSourceCommand.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OverwriteOptions} import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.sources.InsertableRelation @@ -30,7 +30,7 @@ import org.apache.spark.sql.sources.InsertableRelation case class InsertIntoDataSourceCommand( logicalRelation: LogicalRelation, query: LogicalPlan, - overwrite: Boolean) + overwrite: OverwriteOptions) extends RunnableCommand { override protected def innerChildren: Seq[QueryPlan[_]] = Seq(query) @@ -40,7 +40,7 @@ case class InsertIntoDataSourceCommand( val data = Dataset.ofRows(sparkSession, query) // Apply the schema of the existing table to the new data. val df = sparkSession.internalCreateDataFrame(data.queryExecution.toRdd, logicalRelation.schema) - relation.insert(df, overwrite) + relation.insert(df, overwrite.enabled) // Invalidate the cache. sparkSession.sharedState.cacheManager.invalidateCache(logicalRelation) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 9d2930948d6b..ce1e3eb1a5bc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -46,7 +46,8 @@ private[hive] trait HiveStrategies { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.InsertIntoTable( table: MetastoreRelation, partition, child, overwrite, ifNotExists) => - InsertIntoHiveTable(table, partition, planLater(child), overwrite, ifNotExists) :: Nil + InsertIntoHiveTable( + table, partition, planLater(child), overwrite.enabled, ifNotExists) :: Nil case CreateTable(tableDesc, mode, Some(query)) if tableDesc.provider.get == "hive" => val newTableDesc = if (tableDesc.storage.serde.isEmpty) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index ef5a5a001fb6..cac43597aef2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -21,7 +21,7 @@ import scala.util.control.NonFatal import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan, OverwriteOptions} import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.hive.MetastoreRelation @@ -88,7 +88,8 @@ case class CreateHiveTableAsSelectCommand( } else { try { sparkSession.sessionState.executePlan(InsertIntoTable( - metastoreRelation, Map(), query, overwrite = true, ifNotExists = false)).toRdd + metastoreRelation, Map(), query, overwrite = OverwriteOptions(true), + ifNotExists = false)).toRdd } catch { case NonFatal(e) => // drop the created table. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala index 5f16960fb149..ac435bf6195b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/PartitionProviderCompatibilitySuite.scala @@ -134,4 +134,56 @@ class PartitionProviderCompatibilitySuite } } } + + test("insert overwrite partition of legacy datasource table overwrites entire table") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "false") { + withTable("test") { + withTempDir { dir => + setupPartitionedDatasourceTable("test", dir) + spark.sql( + """insert overwrite table test + |partition (partCol=1) + |select * from range(100)""".stripMargin) + assert(spark.sql("select * from test").count() == 100) + + // Dynamic partitions case + spark.sql("insert overwrite table test select id, id from range(10)".stripMargin) + assert(spark.sql("select * from test").count() == 10) + } + } + } + } + + test("insert overwrite partition of new datasource table overwrites just partition") { + withSQLConf(SQLConf.HIVE_MANAGE_FILESOURCE_PARTITIONS.key -> "true") { + withTable("test") { + withTempDir { dir => + setupPartitionedDatasourceTable("test", dir) + sql("msck repair table test") + spark.sql( + """insert overwrite table test + |partition (partCol=1) + |select * from range(100)""".stripMargin) + assert(spark.sql("select * from test").count() == 104) + + // Test overwriting a partition that has a custom location + withTempDir { dir2 => + sql( + s"""alter table test partition (partCol=1) + |set location '${dir2.getAbsolutePath}'""".stripMargin) + assert(sql("select * from test").count() == 4) + sql( + """insert overwrite table test + |partition (partCol=1) + |select * from range(30)""".stripMargin) + sql( + """insert overwrite table test + |partition (partCol=1) + |select * from range(20)""".stripMargin) + assert(sql("select * from test").count() == 24) + } + } + } + } + } } From a36653c5b7b2719f8bfddf4ddfc6e1b828ac9af1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 1 Nov 2016 23:37:03 -0700 Subject: [PATCH 040/132] [SPARK-18192] Support all file formats in structured streaming ## What changes were proposed in this pull request? This patch adds support for all file formats in structured streaming sinks. This is actually a very small change thanks to all the previous refactoring done using the new internal commit protocol API. ## How was this patch tested? Updated FileStreamSinkSuite to add test cases for json, text, and parquet. Author: Reynold Xin Closes #15711 from rxin/SPARK-18192. --- .../execution/datasources/DataSource.scala | 8 +-- .../sql/streaming/FileStreamSinkSuite.scala | 62 +++++++++---------- 2 files changed, 32 insertions(+), 38 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index d980e6a15aab..3f956c427655 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -29,7 +29,6 @@ import org.apache.hadoop.fs.Path import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat @@ -37,7 +36,6 @@ 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.parquet.ParquetFileFormat import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{CalendarIntervalType, StructType} @@ -292,7 +290,7 @@ case class DataSource( case s: StreamSinkProvider => s.createSink(sparkSession.sqlContext, options, partitionColumns, outputMode) - case parquet: parquet.ParquetFileFormat => + case fileFormat: FileFormat => val caseInsensitiveOptions = new CaseInsensitiveMap(options) val path = caseInsensitiveOptions.getOrElse("path", { throw new IllegalArgumentException("'path' is not specified") @@ -301,7 +299,7 @@ case class DataSource( throw new IllegalArgumentException( s"Data source $className does not support $outputMode output mode") } - new FileStreamSink(sparkSession, path, parquet, partitionColumns, options) + new FileStreamSink(sparkSession, path, fileFormat, partitionColumns, options) case _ => throw new UnsupportedOperationException( @@ -516,7 +514,7 @@ case class DataSource( val plan = data.logicalPlan plan.resolve(name :: Nil, data.sparkSession.sessionState.analyzer.resolver).getOrElse { throw new AnalysisException( - s"Unable to resolve ${name} given [${plan.output.map(_.name).mkString(", ")}]") + s"Unable to resolve $name given [${plan.output.map(_.name).mkString(", ")}]") }.asInstanceOf[Attribute] } // For partitioned relation r, r.schema's column ordering can be different from the column diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 902cf0534471..0f140f94f630 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.streaming -import org.apache.spark.sql._ +import org.apache.spark.sql.DataFrame import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.streaming.{MemoryStream, MetadataLogFileIndex} @@ -142,42 +142,38 @@ class FileStreamSinkSuite extends StreamTest { } } - test("FileStreamSink - supported formats") { - def testFormat(format: Option[String]): Unit = { - val inputData = MemoryStream[Int] - val ds = inputData.toDS() + test("FileStreamSink - parquet") { + testFormat(None) // should not throw error as default format parquet when not specified + testFormat(Some("parquet")) + } - val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath - val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + test("FileStreamSink - text") { + testFormat(Some("text")) + } - var query: StreamingQuery = null + test("FileStreamSink - json") { + testFormat(Some("text")) + } - try { - val writer = - ds.map(i => (i, i * 1000)) - .toDF("id", "value") - .writeStream - if (format.nonEmpty) { - writer.format(format.get) - } - query = writer - .option("checkpointLocation", checkpointDir) - .start(outputDir) - } finally { - if (query != null) { - query.stop() - } - } - } + def testFormat(format: Option[String]): Unit = { + val inputData = MemoryStream[Int] + val ds = inputData.toDS() - testFormat(None) // should not throw error as default format parquet when not specified - testFormat(Some("parquet")) - val e = intercept[UnsupportedOperationException] { - testFormat(Some("text")) - } - Seq("text", "not support", "stream").foreach { s => - assert(e.getMessage.contains(s)) + val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath + val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath + + var query: StreamingQuery = null + + try { + val writer = ds.map(i => (i, i * 1000)).toDF("id", "value").writeStream + if (format.nonEmpty) { + writer.format(format.get) + } + query = writer.option("checkpointLocation", checkpointDir).start(outputDir) + } finally { + if (query != null) { + query.stop() + } } } - } From 85c5424d466f4a5765c825e0e2ab30da97611285 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 1 Nov 2016 23:39:53 -0700 Subject: [PATCH 041/132] [SPARK-18144][SQL] logging StreamingQueryListener$QueryStartedEvent ## What changes were proposed in this pull request? The PR fixes the bug that the QueryStartedEvent is not logged the postToAll() in the original code is actually calling StreamingQueryListenerBus.postToAll() which has no listener at all....we shall post by sparkListenerBus.postToAll(s) and this.postToAll() to trigger local listeners as well as the listeners registered in LiveListenerBus zsxwing ## How was this patch tested? The following snapshot shows that QueryStartedEvent has been logged correctly ![image](https://cloud.githubusercontent.com/assets/678008/19821553/007a7d28-9d2d-11e6-9f13-49851559cdaa.png) Author: CodingCat Closes #15675 from CodingCat/SPARK-18144. --- .../streaming/StreamingQueryListenerBus.scala | 10 +++++++++- .../spark/sql/streaming/StreamingQuerySuite.scala | 7 ++++++- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala index fc2190d39da4..22e4c6380fcd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingQueryListenerBus.scala @@ -41,6 +41,8 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) def post(event: StreamingQueryListener.Event) { event match { case s: QueryStartedEvent => + sparkListenerBus.post(s) + // post to local listeners to trigger callbacks postToAll(s) case _ => sparkListenerBus.post(event) @@ -50,7 +52,13 @@ class StreamingQueryListenerBus(sparkListenerBus: LiveListenerBus) override def onOtherEvent(event: SparkListenerEvent): Unit = { event match { case e: StreamingQueryListener.Event => - postToAll(e) + // SPARK-18144: we broadcast QueryStartedEvent to all listeners attached to this bus + // synchronously and the ones attached to LiveListenerBus asynchronously. Therefore, + // we need to ignore QueryStartedEvent if this method is called within SparkListenerBus + // thread + if (!LiveListenerBus.withinListenerThread.value || !e.isInstanceOf[QueryStartedEvent]) { + postToAll(e) + } case _ => } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 464c443beb6e..31b7fe0b04da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -290,7 +290,10 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { // A StreamingQueryListener that gets the query status after the first completed trigger val listener = new StreamingQueryListener { @volatile var firstStatus: StreamingQueryStatus = null - override def onQueryStarted(queryStarted: QueryStartedEvent): Unit = { } + @volatile var queryStartedEvent = 0 + override def onQueryStarted(queryStarted: QueryStartedEvent): Unit = { + queryStartedEvent += 1 + } override def onQueryProgress(queryProgress: QueryProgressEvent): Unit = { if (firstStatus == null) firstStatus = queryProgress.queryStatus } @@ -303,6 +306,8 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { q.processAllAvailable() eventually(timeout(streamingTimeout)) { assert(listener.firstStatus != null) + // test if QueryStartedEvent callback is called for only once + assert(listener.queryStartedEvent === 1) } listener.firstStatus } finally { From 2dc048081668665f85623839d5f663b402e42555 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Wed, 2 Nov 2016 00:08:30 -0700 Subject: [PATCH 042/132] [SPARK-17532] Add lock debugging info to thread dumps. ## What changes were proposed in this pull request? This adds information to the web UI thread dump page about the JVM locks held by threads and the locks that threads are blocked waiting to acquire. This should help find cases where lock contention is causing Spark applications to run slowly. ## How was this patch tested? Tested by applying this patch and viewing the change in the web UI. ![thread-lock-info](https://cloud.githubusercontent.com/assets/87915/18493057/6e5da870-79c3-11e6-8c20-f54c18a37544.png) Additions: - A "Thread Locking" column with the locks held by the thread or that are blocking the thread - Links from the a blocked thread to the thread holding the lock - Stack frames show where threads are inside `synchronized` blocks, "holding Monitor(...)" Author: Ryan Blue Closes #15088 from rdblue/SPARK-17532-add-thread-lock-info. --- .../org/apache/spark/ui/static/table.js | 3 +- .../ui/exec/ExecutorThreadDumpPage.scala | 12 +++++++ .../apache/spark/util/ThreadStackTrace.scala | 6 +++- .../scala/org/apache/spark/util/Utils.scala | 34 ++++++++++++++++--- 4 files changed, 49 insertions(+), 6 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/table.js b/core/src/main/resources/org/apache/spark/ui/static/table.js index 14b06bfe860e..0315ebf5c48a 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/table.js +++ b/core/src/main/resources/org/apache/spark/ui/static/table.js @@ -36,7 +36,7 @@ function toggleThreadStackTrace(threadId, forceAdd) { if (stackTrace.length == 0) { var stackTraceText = $('#' + threadId + "_td_stacktrace").html() var threadCell = $("#thread_" + threadId + "_tr") - threadCell.after("
" +
+        threadCell.after("
" +
             stackTraceText +  "
") } else { if (!forceAdd) { @@ -73,6 +73,7 @@ function onMouseOverAndOut(threadId) { $("#" + threadId + "_td_id").toggleClass("threaddump-td-mouseover"); $("#" + threadId + "_td_name").toggleClass("threaddump-td-mouseover"); $("#" + threadId + "_td_state").toggleClass("threaddump-td-mouseover"); + $("#" + threadId + "_td_locking").toggleClass("threaddump-td-mouseover"); } function onSearchStringChange() { diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala index a0ef80d9bdae..c6a07445f2a3 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala @@ -48,6 +48,16 @@ private[ui] class ExecutorThreadDumpPage(parent: ExecutorsTab) extends WebUIPage } }.map { thread => val threadId = thread.threadId + val blockedBy = thread.blockedByThreadId match { + case Some(blockedByThreadId) => + + case None => Text("") + } + val heldLocks = thread.holdingLocks.mkString(", ") + {threadId} {thread.threadName} {thread.threadState} + {blockedBy}{heldLocks} {thread.stackTrace} } @@ -86,6 +97,7 @@ private[ui] class ExecutorThreadDumpPage(parent: ExecutorsTab) extends WebUIPage Thread ID Thread Name Thread State + Thread Locks {dumpRows} diff --git a/core/src/main/scala/org/apache/spark/util/ThreadStackTrace.scala b/core/src/main/scala/org/apache/spark/util/ThreadStackTrace.scala index d4e0ad93b966..b1217980faf1 100644 --- a/core/src/main/scala/org/apache/spark/util/ThreadStackTrace.scala +++ b/core/src/main/scala/org/apache/spark/util/ThreadStackTrace.scala @@ -24,4 +24,8 @@ private[spark] case class ThreadStackTrace( threadId: Long, threadName: String, threadState: Thread.State, - stackTrace: String) + stackTrace: String, + blockedByThreadId: Option[Long], + blockedByLock: String, + holdingLocks: Seq[String]) + diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 6027b07c0fee..22c28fba2087 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -18,7 +18,7 @@ package org.apache.spark.util import java.io._ -import java.lang.management.ManagementFactory +import java.lang.management.{LockInfo, ManagementFactory, MonitorInfo} import java.net._ import java.nio.ByteBuffer import java.nio.channels.Channels @@ -2096,15 +2096,41 @@ private[spark] object Utils extends Logging { } } + private implicit class Lock(lock: LockInfo) { + def lockString: String = { + lock match { + case monitor: MonitorInfo => + s"Monitor(${lock.getClassName}@${lock.getIdentityHashCode}})" + case _ => + s"Lock(${lock.getClassName}@${lock.getIdentityHashCode}})" + } + } + } + /** Return a thread dump of all threads' stacktraces. Used to capture dumps for the web UI */ def getThreadDump(): Array[ThreadStackTrace] = { // We need to filter out null values here because dumpAllThreads() may return null array // elements for threads that are dead / don't exist. val threadInfos = ManagementFactory.getThreadMXBean.dumpAllThreads(true, true).filter(_ != null) threadInfos.sortBy(_.getThreadId).map { case threadInfo => - val stackTrace = threadInfo.getStackTrace.map(_.toString).mkString("\n") - ThreadStackTrace(threadInfo.getThreadId, threadInfo.getThreadName, - threadInfo.getThreadState, stackTrace) + val monitors = threadInfo.getLockedMonitors.map(m => m.getLockedStackFrame -> m).toMap + val stackTrace = threadInfo.getStackTrace.map { frame => + monitors.get(frame) match { + case Some(monitor) => + monitor.getLockedStackFrame.toString + s" => holding ${monitor.lockString}" + case None => + frame.toString + } + }.mkString("\n") + + // use a set to dedup re-entrant locks that are held at multiple places + val heldLocks = (threadInfo.getLockedSynchronizers.map(_.lockString) + ++ threadInfo.getLockedMonitors.map(_.lockString) + ).toSet + + ThreadStackTrace(threadInfo.getThreadId, threadInfo.getThreadName, threadInfo.getThreadState, + stackTrace, if (threadInfo.getLockOwnerId < 0) None else Some(threadInfo.getLockOwnerId), + Option(threadInfo.getLockInfo).map(_.lockString).getOrElse(""), heldLocks.toSeq) } } From bcbe44440e6c871e217f06d2a4696fd41f1d2606 Mon Sep 17 00:00:00 2001 From: Maria Rydzy Date: Wed, 2 Nov 2016 09:09:16 +0000 Subject: [PATCH 043/132] [MINOR] Use <= for clarity in Pi examples' Monte Carlo process ## What changes were proposed in this pull request? If my understanding is correct we should be rather looking at closed disk than the opened one. ## How was this patch tested? Run simple comparison, of the mean squared error of approaches with closed and opened disk. https://gist.github.com/mrydzy/1cf0e5c316ef9d6fbd91426b91f1969f The closed one performed slightly better, but the tested sample wasn't too big, so I rely mostly on the algorithm understanding. Author: Maria Rydzy Closes #15687 from mrydzy/master. --- .../src/main/java/org/apache/spark/examples/JavaSparkPi.java | 2 +- examples/src/main/python/pi.py | 2 +- examples/src/main/scala/org/apache/spark/examples/LocalPi.scala | 2 +- examples/src/main/scala/org/apache/spark/examples/SparkPi.scala | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index 7df145e3117b..89855e81f1f7 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -54,7 +54,7 @@ public static void main(String[] args) throws Exception { public Integer call(Integer integer) { double x = Math.random() * 2 - 1; double y = Math.random() * 2 - 1; - return (x * x + y * y < 1) ? 1 : 0; + return (x * x + y * y <= 1) ? 1 : 0; } }).reduce(new Function2() { @Override diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index e3f0c4aeef1b..37029b76798f 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -38,7 +38,7 @@ def f(_): x = random() * 2 - 1 y = random() * 2 - 1 - return 1 if x ** 2 + y ** 2 < 1 else 0 + return 1 if x ** 2 + y ** 2 <= 1 else 0 count = spark.sparkContext.parallelize(range(1, n + 1), partitions).map(f).reduce(add) print("Pi is roughly %f" % (4.0 * count / n)) diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala index 720d92fb9d02..121b768e4198 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala @@ -26,7 +26,7 @@ object LocalPi { for (i <- 1 to 100000) { val x = random * 2 - 1 val y = random * 2 - 1 - if (x*x + y*y < 1) count += 1 + if (x*x + y*y <= 1) count += 1 } println("Pi is roughly " + 4 * count / 100000.0) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index 272c1a4fc2f4..a5cacf17a5cc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -34,7 +34,7 @@ object SparkPi { val count = spark.sparkContext.parallelize(1 until n, slices).map { i => val x = random * 2 - 1 val y = random * 2 - 1 - if (x*x + y*y < 1) 1 else 0 + if (x*x + y*y <= 1) 1 else 0 }.reduce(_ + _) println("Pi is roughly " + 4.0 * count / (n - 1)) spark.stop() From 98ede49496d0d7b4724085083d4f24436b92a7bf Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Wed, 2 Nov 2016 09:10:34 +0000 Subject: [PATCH 044/132] [SPARK-18198][DOC][STREAMING] Highlight code snippets ## What changes were proposed in this pull request? This patch uses `{% highlight lang %}...{% endhighlight %}` to highlight code snippets in the `Structured Streaming Kafka010 integration doc` and the `Spark Streaming Kafka010 integration doc`. This patch consists of two commits: - the first commit fixes only the leading spaces -- this is large - the second commit adds the highlight instructions -- this is much simpler and easier to review ## How was this patch tested? SKIP_API=1 jekyll build ## Screenshots **Before** ![snip20161101_3](https://cloud.githubusercontent.com/assets/15843379/19894258/47746524-a087-11e6-9a2a-7bff2d428d44.png) **After** ![snip20161101_1](https://cloud.githubusercontent.com/assets/15843379/19894324/8bebcd1e-a087-11e6-835b-88c4d2979cfa.png) Author: Liwei Lin Closes #15715 from lw-lin/doc-highlight-code-snippet. --- docs/streaming-kafka-0-10-integration.md | 391 +++++++++--------- .../structured-streaming-kafka-integration.md | 156 +++---- 2 files changed, 287 insertions(+), 260 deletions(-) diff --git a/docs/streaming-kafka-0-10-integration.md b/docs/streaming-kafka-0-10-integration.md index c1ef396907db..b645d3c3a4b5 100644 --- a/docs/streaming-kafka-0-10-integration.md +++ b/docs/streaming-kafka-0-10-integration.md @@ -17,69 +17,72 @@ For Scala/Java applications using SBT/Maven project definitions, link your strea
- import org.apache.kafka.clients.consumer.ConsumerRecord - import org.apache.kafka.common.serialization.StringDeserializer - import org.apache.spark.streaming.kafka010._ - import org.apache.spark.streaming.kafka010.LocationStrategies.PreferConsistent - import org.apache.spark.streaming.kafka010.ConsumerStrategies.Subscribe - - val kafkaParams = Map[String, Object]( - "bootstrap.servers" -> "localhost:9092,anotherhost:9092", - "key.deserializer" -> classOf[StringDeserializer], - "value.deserializer" -> classOf[StringDeserializer], - "group.id" -> "use_a_separate_group_id_for_each_stream", - "auto.offset.reset" -> "latest", - "enable.auto.commit" -> (false: java.lang.Boolean) - ) - - val topics = Array("topicA", "topicB") - val stream = KafkaUtils.createDirectStream[String, String]( - streamingContext, - PreferConsistent, - Subscribe[String, String](topics, kafkaParams) - ) - - stream.map(record => (record.key, record.value)) - +{% highlight scala %} +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.common.serialization.StringDeserializer +import org.apache.spark.streaming.kafka010._ +import org.apache.spark.streaming.kafka010.LocationStrategies.PreferConsistent +import org.apache.spark.streaming.kafka010.ConsumerStrategies.Subscribe + +val kafkaParams = Map[String, Object]( + "bootstrap.servers" -> "localhost:9092,anotherhost:9092", + "key.deserializer" -> classOf[StringDeserializer], + "value.deserializer" -> classOf[StringDeserializer], + "group.id" -> "use_a_separate_group_id_for_each_stream", + "auto.offset.reset" -> "latest", + "enable.auto.commit" -> (false: java.lang.Boolean) +) + +val topics = Array("topicA", "topicB") +val stream = KafkaUtils.createDirectStream[String, String]( + streamingContext, + PreferConsistent, + Subscribe[String, String](topics, kafkaParams) +) + +stream.map(record => (record.key, record.value)) +{% endhighlight %} Each item in the stream is a [ConsumerRecord](http://kafka.apache.org/0100/javadoc/org/apache/kafka/clients/consumer/ConsumerRecord.html)
- import java.util.*; - import org.apache.spark.SparkConf; - import org.apache.spark.TaskContext; - import org.apache.spark.api.java.*; - import org.apache.spark.api.java.function.*; - import org.apache.spark.streaming.api.java.*; - import org.apache.spark.streaming.kafka010.*; - import org.apache.kafka.clients.consumer.ConsumerRecord; - import org.apache.kafka.common.TopicPartition; - import org.apache.kafka.common.serialization.StringDeserializer; - import scala.Tuple2; - - Map kafkaParams = new HashMap<>(); - kafkaParams.put("bootstrap.servers", "localhost:9092,anotherhost:9092"); - kafkaParams.put("key.deserializer", StringDeserializer.class); - kafkaParams.put("value.deserializer", StringDeserializer.class); - kafkaParams.put("group.id", "use_a_separate_group_id_for_each_stream"); - kafkaParams.put("auto.offset.reset", "latest"); - kafkaParams.put("enable.auto.commit", false); - - Collection topics = Arrays.asList("topicA", "topicB"); - - final JavaInputDStream> stream = - KafkaUtils.createDirectStream( - streamingContext, - LocationStrategies.PreferConsistent(), - ConsumerStrategies.Subscribe(topics, kafkaParams) - ); - - stream.mapToPair( - new PairFunction, String, String>() { - @Override - public Tuple2 call(ConsumerRecord record) { - return new Tuple2<>(record.key(), record.value()); - } - }) +{% highlight java %} +import java.util.*; +import org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.*; +import org.apache.spark.streaming.api.java.*; +import org.apache.spark.streaming.kafka010.*; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.StringDeserializer; +import scala.Tuple2; + +Map kafkaParams = new HashMap<>(); +kafkaParams.put("bootstrap.servers", "localhost:9092,anotherhost:9092"); +kafkaParams.put("key.deserializer", StringDeserializer.class); +kafkaParams.put("value.deserializer", StringDeserializer.class); +kafkaParams.put("group.id", "use_a_separate_group_id_for_each_stream"); +kafkaParams.put("auto.offset.reset", "latest"); +kafkaParams.put("enable.auto.commit", false); + +Collection topics = Arrays.asList("topicA", "topicB"); + +final JavaInputDStream> stream = + KafkaUtils.createDirectStream( + streamingContext, + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Subscribe(topics, kafkaParams) + ); + +stream.mapToPair( + new PairFunction, String, String>() { + @Override + public Tuple2 call(ConsumerRecord record) { + return new Tuple2<>(record.key(), record.value()); + } + }) +{% endhighlight %}
@@ -109,32 +112,35 @@ If you have a use case that is better suited to batch processing, you can create
- // Import dependencies and create kafka params as in Create Direct Stream above - - val offsetRanges = Array( - // topic, partition, inclusive starting offset, exclusive ending offset - OffsetRange("test", 0, 0, 100), - OffsetRange("test", 1, 0, 100) - ) +{% highlight scala %} +// Import dependencies and create kafka params as in Create Direct Stream above - val rdd = KafkaUtils.createRDD[String, String](sparkContext, kafkaParams, offsetRanges, PreferConsistent) +val offsetRanges = Array( + // topic, partition, inclusive starting offset, exclusive ending offset + OffsetRange("test", 0, 0, 100), + OffsetRange("test", 1, 0, 100) +) +val rdd = KafkaUtils.createRDD[String, String](sparkContext, kafkaParams, offsetRanges, PreferConsistent) +{% endhighlight %}
- // Import dependencies and create kafka params as in Create Direct Stream above - - OffsetRange[] offsetRanges = { - // topic, partition, inclusive starting offset, exclusive ending offset - OffsetRange.create("test", 0, 0, 100), - OffsetRange.create("test", 1, 0, 100) - }; - - JavaRDD> rdd = KafkaUtils.createRDD( - sparkContext, - kafkaParams, - offsetRanges, - LocationStrategies.PreferConsistent() - ); +{% highlight java %} +// Import dependencies and create kafka params as in Create Direct Stream above + +OffsetRange[] offsetRanges = { + // topic, partition, inclusive starting offset, exclusive ending offset + OffsetRange.create("test", 0, 0, 100), + OffsetRange.create("test", 1, 0, 100) +}; + +JavaRDD> rdd = KafkaUtils.createRDD( + sparkContext, + kafkaParams, + offsetRanges, + LocationStrategies.PreferConsistent() +); +{% endhighlight %}
@@ -144,29 +150,33 @@ Note that you cannot use `PreferBrokers`, because without the stream there is no
- stream.foreachRDD { rdd => - val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges - rdd.foreachPartition { iter => - val o: OffsetRange = offsetRanges(TaskContext.get.partitionId) - println(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") - } - } +{% highlight scala %} +stream.foreachRDD { rdd => + val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + rdd.foreachPartition { iter => + val o: OffsetRange = offsetRanges(TaskContext.get.partitionId) + println(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") + } +} +{% endhighlight %}
- stream.foreachRDD(new VoidFunction>>() { - @Override - public void call(JavaRDD> rdd) { - final OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); - rdd.foreachPartition(new VoidFunction>>() { - @Override - public void call(Iterator> consumerRecords) { - OffsetRange o = offsetRanges[TaskContext.get().partitionId()]; - System.out.println( - o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset()); - } - }); - } - }); +{% highlight java %} +stream.foreachRDD(new VoidFunction>>() { + @Override + public void call(JavaRDD> rdd) { + final OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + rdd.foreachPartition(new VoidFunction>>() { + @Override + public void call(Iterator> consumerRecords) { + OffsetRange o = offsetRanges[TaskContext.get().partitionId()]; + System.out.println( + o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset()); + } + }); + } +}); +{% endhighlight %}
@@ -183,25 +193,28 @@ Kafka has an offset commit API that stores offsets in a special Kafka topic. By
- stream.foreachRDD { rdd => - val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges - - // some time later, after outputs have completed - stream.asInstanceOf[CanCommitOffsets].commitAsync(offsetRanges) - } - +{% highlight scala %} +stream.foreachRDD { rdd => + val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + + // some time later, after outputs have completed + stream.asInstanceOf[CanCommitOffsets].commitAsync(offsetRanges) +} +{% endhighlight %} As with HasOffsetRanges, the cast to CanCommitOffsets will only succeed if called on the result of createDirectStream, not after transformations. The commitAsync call is threadsafe, but must occur after outputs if you want meaningful semantics.
- stream.foreachRDD(new VoidFunction>>() { - @Override - public void call(JavaRDD> rdd) { - OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); - - // some time later, after outputs have completed - ((CanCommitOffsets) stream.inputDStream()).commitAsync(offsetRanges); - } - }); +{% highlight java %} +stream.foreachRDD(new VoidFunction>>() { + @Override + public void call(JavaRDD> rdd) { + OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + + // some time later, after outputs have completed + ((CanCommitOffsets) stream.inputDStream()).commitAsync(offsetRanges); + } +}); +{% endhighlight %}
@@ -210,64 +223,68 @@ For data stores that support transactions, saving offsets in the same transactio
- // The details depend on your data store, but the general idea looks like this +{% highlight scala %} +// The details depend on your data store, but the general idea looks like this - // begin from the the offsets committed to the database - val fromOffsets = selectOffsetsFromYourDatabase.map { resultSet => - new TopicPartition(resultSet.string("topic"), resultSet.int("partition")) -> resultSet.long("offset") - }.toMap +// begin from the the offsets committed to the database +val fromOffsets = selectOffsetsFromYourDatabase.map { resultSet => + new TopicPartition(resultSet.string("topic"), resultSet.int("partition")) -> resultSet.long("offset") +}.toMap - val stream = KafkaUtils.createDirectStream[String, String]( - streamingContext, - PreferConsistent, - Assign[String, String](fromOffsets.keys.toList, kafkaParams, fromOffsets) - ) +val stream = KafkaUtils.createDirectStream[String, String]( + streamingContext, + PreferConsistent, + Assign[String, String](fromOffsets.keys.toList, kafkaParams, fromOffsets) +) - stream.foreachRDD { rdd => - val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges +stream.foreachRDD { rdd => + val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges - val results = yourCalculation(rdd) + val results = yourCalculation(rdd) - // begin your transaction + // begin your transaction - // update results - // update offsets where the end of existing offsets matches the beginning of this batch of offsets - // assert that offsets were updated correctly + // update results + // update offsets where the end of existing offsets matches the beginning of this batch of offsets + // assert that offsets were updated correctly - // end your transaction - } + // end your transaction +} +{% endhighlight %}
- // The details depend on your data store, but the general idea looks like this - - // begin from the the offsets committed to the database - Map fromOffsets = new HashMap<>(); - for (resultSet : selectOffsetsFromYourDatabase) - fromOffsets.put(new TopicPartition(resultSet.string("topic"), resultSet.int("partition")), resultSet.long("offset")); - } - - JavaInputDStream> stream = KafkaUtils.createDirectStream( - streamingContext, - LocationStrategies.PreferConsistent(), - ConsumerStrategies.Assign(fromOffsets.keySet(), kafkaParams, fromOffsets) - ); - - stream.foreachRDD(new VoidFunction>>() { - @Override - public void call(JavaRDD> rdd) { - OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); - - Object results = yourCalculation(rdd); - - // begin your transaction - - // update results - // update offsets where the end of existing offsets matches the beginning of this batch of offsets - // assert that offsets were updated correctly - - // end your transaction - } - }); +{% highlight java %} +// The details depend on your data store, but the general idea looks like this + +// begin from the the offsets committed to the database +Map fromOffsets = new HashMap<>(); +for (resultSet : selectOffsetsFromYourDatabase) + fromOffsets.put(new TopicPartition(resultSet.string("topic"), resultSet.int("partition")), resultSet.long("offset")); +} + +JavaInputDStream> stream = KafkaUtils.createDirectStream( + streamingContext, + LocationStrategies.PreferConsistent(), + ConsumerStrategies.Assign(fromOffsets.keySet(), kafkaParams, fromOffsets) +); + +stream.foreachRDD(new VoidFunction>>() { + @Override + public void call(JavaRDD> rdd) { + OffsetRange[] offsetRanges = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + + Object results = yourCalculation(rdd); + + // begin your transaction + + // update results + // update offsets where the end of existing offsets matches the beginning of this batch of offsets + // assert that offsets were updated correctly + + // end your transaction + } +}); +{% endhighlight %}
@@ -277,25 +294,29 @@ The new Kafka consumer [supports SSL](http://kafka.apache.org/documentation.html
- val kafkaParams = Map[String, Object]( - // the usual params, make sure to change the port in bootstrap.servers if 9092 is not TLS - "security.protocol" -> "SSL", - "ssl.truststore.location" -> "/some-directory/kafka.client.truststore.jks", - "ssl.truststore.password" -> "test1234", - "ssl.keystore.location" -> "/some-directory/kafka.client.keystore.jks", - "ssl.keystore.password" -> "test1234", - "ssl.key.password" -> "test1234" - ) +{% highlight scala %} +val kafkaParams = Map[String, Object]( + // the usual params, make sure to change the port in bootstrap.servers if 9092 is not TLS + "security.protocol" -> "SSL", + "ssl.truststore.location" -> "/some-directory/kafka.client.truststore.jks", + "ssl.truststore.password" -> "test1234", + "ssl.keystore.location" -> "/some-directory/kafka.client.keystore.jks", + "ssl.keystore.password" -> "test1234", + "ssl.key.password" -> "test1234" +) +{% endhighlight %}
- Map kafkaParams = new HashMap(); - // the usual params, make sure to change the port in bootstrap.servers if 9092 is not TLS - kafkaParams.put("security.protocol", "SSL"); - kafkaParams.put("ssl.truststore.location", "/some-directory/kafka.client.truststore.jks"); - kafkaParams.put("ssl.truststore.password", "test1234"); - kafkaParams.put("ssl.keystore.location", "/some-directory/kafka.client.keystore.jks"); - kafkaParams.put("ssl.keystore.password", "test1234"); - kafkaParams.put("ssl.key.password", "test1234"); +{% highlight java %} +Map kafkaParams = new HashMap(); +// the usual params, make sure to change the port in bootstrap.servers if 9092 is not TLS +kafkaParams.put("security.protocol", "SSL"); +kafkaParams.put("ssl.truststore.location", "/some-directory/kafka.client.truststore.jks"); +kafkaParams.put("ssl.truststore.password", "test1234"); +kafkaParams.put("ssl.keystore.location", "/some-directory/kafka.client.keystore.jks"); +kafkaParams.put("ssl.keystore.password", "test1234"); +kafkaParams.put("ssl.key.password", "test1234"); +{% endhighlight %}
diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index a6c3b3a9024d..c4c9fb3f7d3d 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -19,97 +19,103 @@ application. See the [Deploying](#deploying) subsection below.
+{% highlight scala %} - // Subscribe to 1 topic - val ds1 = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("subscribe", "topic1") - .load() - ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .as[(String, String)] +// Subscribe to 1 topic +val ds1 = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() +ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] - // Subscribe to multiple topics - val ds2 = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("subscribe", "topic1,topic2") - .load() - ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .as[(String, String)] +// Subscribe to multiple topics +val ds2 = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1,topic2") + .load() +ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] - // Subscribe to a pattern - val ds3 = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("subscribePattern", "topic.*") - .load() - ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - .as[(String, String)] +// Subscribe to a pattern +val ds3 = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .load() +ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] +{% endhighlight %}
+{% highlight java %} - // Subscribe to 1 topic - Dataset ds1 = spark - .readStream() - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("subscribe", "topic1") - .load() - ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") +// Subscribe to 1 topic +Dataset ds1 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() +ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - // Subscribe to multiple topics - Dataset ds2 = spark - .readStream() - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("subscribe", "topic1,topic2") - .load() - ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") +// Subscribe to multiple topics +Dataset ds2 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1,topic2") + .load() +ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - // Subscribe to a pattern - Dataset ds3 = spark - .readStream() - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("subscribePattern", "topic.*") - .load() - ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") +// Subscribe to a pattern +Dataset ds3 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .load() +ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") +{% endhighlight %}
+{% highlight python %} - # Subscribe to 1 topic - ds1 = spark - .readStream() - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("subscribe", "topic1") - .load() - ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") +# Subscribe to 1 topic +ds1 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1") + .load() +ds1.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - # Subscribe to multiple topics - ds2 = spark - .readStream - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("subscribe", "topic1,topic2") - .load() - ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") +# Subscribe to multiple topics +ds2 = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribe", "topic1,topic2") + .load() +ds2.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") - # Subscribe to a pattern - ds3 = spark - .readStream() - .format("kafka") - .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .option("subscribePattern", "topic.*") - .load() - ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") +# Subscribe to a pattern +ds3 = spark + .readStream() + .format("kafka") + .option("kafka.bootstrap.servers", "host1:port1,host2:port2") + .option("subscribePattern", "topic.*") + .load() +ds3.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") +{% endhighlight %}
From 70a5db7bbd192a4bc68bcfdc475ab221adf2fcdd Mon Sep 17 00:00:00 2001 From: Jacek Laskowski Date: Wed, 2 Nov 2016 09:21:26 +0000 Subject: [PATCH 045/132] [SPARK-18204][WEBUI] Remove SparkUI.appUIAddress ## What changes were proposed in this pull request? Removing `appUIAddress` attribute since it is no longer in use. ## How was this patch tested? Local build Author: Jacek Laskowski Closes #15603 from jaceklaskowski/sparkui-fixes. --- .../cluster/StandaloneSchedulerBackend.scala | 6 +++--- .../main/scala/org/apache/spark/ui/SparkUI.scala | 13 +++---------- .../main/scala/org/apache/spark/ui/WebUI.scala | 8 ++++---- .../org/apache/spark/ui/jobs/AllJobsPage.scala | 4 ++-- .../org/apache/spark/ui/UISeleniumSuite.scala | 16 ++++++++-------- .../test/scala/org/apache/spark/ui/UISuite.scala | 13 ++++++------- .../MesosCoarseGrainedSchedulerBackend.scala | 2 +- .../mesos/MesosFineGrainedSchedulerBackend.scala | 2 +- .../apache/spark/streaming/UISeleniumSuite.scala | 12 ++++++------ .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- .../cluster/YarnClientSchedulerBackend.scala | 2 +- 11 files changed, 36 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 04d40e2907cf..368cd30a2e11 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -93,7 +93,7 @@ private[spark] class StandaloneSchedulerBackend( val javaOpts = sparkJavaOpts ++ extraJavaOpts val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries ++ testingClassPath, libraryPathEntries, javaOpts) - val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") + val webUrl = sc.ui.map(_.webUrl).getOrElse("") val coresPerExecutor = conf.getOption("spark.executor.cores").map(_.toInt) // If we're using dynamic allocation, set our initial executor limit to 0 for now. // ExecutorAllocationManager will send the real initial limit to the Master later. @@ -103,8 +103,8 @@ private[spark] class StandaloneSchedulerBackend( } else { None } - val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - appUIAddress, sc.eventLogDir, sc.eventLogCodec, coresPerExecutor, initialExecutorLimit) + val appDesc = ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, + webUrl, sc.eventLogDir, sc.eventLogCodec, coresPerExecutor, initialExecutorLimit) client = new StandaloneAppClient(sc.env.rpcEnv, masters, appDesc, this, conf) client.start() launcherBackend.setState(SparkAppHandle.State.SUBMITTED) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index f631a047a707..b828532aba7a 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -82,7 +82,7 @@ private[spark] class SparkUI private ( initialize() def getSparkUser: String = { - environmentListener.systemProperties.toMap.get("user.name").getOrElse("") + environmentListener.systemProperties.toMap.getOrElse("user.name", "") } def getAppName: String = appName @@ -94,16 +94,9 @@ private[spark] class SparkUI private ( /** Stop the server behind this web interface. Only valid after bind(). */ override def stop() { super.stop() - logInfo("Stopped Spark web UI at %s".format(appUIAddress)) + logInfo(s"Stopped Spark web UI at $webUrl") } - /** - * Return the application UI host:port. This does not include the scheme (http://). - */ - private[spark] def appUIHostPort = publicHostName + ":" + boundPort - - private[spark] def appUIAddress = s"http://$appUIHostPort" - def getSparkUI(appId: String): Option[SparkUI] = { if (appId == this.appId) Some(this) else None } @@ -136,7 +129,7 @@ private[spark] class SparkUI private ( private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String) extends WebUITab(parent, prefix) { - def appName: String = parent.getAppName + def appName: String = parent.appName } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index a05e0efb7a3e..8c801558672f 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -56,8 +56,8 @@ private[spark] abstract class WebUI( private val className = Utils.getFormattedClassName(this) def getBasePath: String = basePath - def getTabs: Seq[WebUITab] = tabs.toSeq - def getHandlers: Seq[ServletContextHandler] = handlers.toSeq + def getTabs: Seq[WebUITab] = tabs + def getHandlers: Seq[ServletContextHandler] = handlers def getSecurityManager: SecurityManager = securityManager /** Attach a tab to this UI, along with all of its attached pages. */ @@ -133,7 +133,7 @@ private[spark] abstract class WebUI( def initialize(): Unit /** Bind to the HTTP server behind this web interface. */ - def bind() { + def bind(): Unit = { assert(!serverInfo.isDefined, s"Attempted to bind $className more than once!") try { val host = Option(conf.getenv("SPARK_LOCAL_IP")).getOrElse("0.0.0.0") @@ -156,7 +156,7 @@ private[spark] abstract class WebUI( def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) /** Stop the server behind this web interface. Only valid after bind(). */ - def stop() { + def stop(): Unit = { assert(serverInfo.isDefined, s"Attempted to stop $className before binding to a server!") serverInfo.get.stop() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index 173fc3cf31ce..50e8e2d19e15 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -289,8 +289,8 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val startTime = listener.startTime val endTime = listener.endTime val activeJobs = listener.activeJobs.values.toSeq - val completedJobs = listener.completedJobs.reverse.toSeq - val failedJobs = listener.failedJobs.reverse.toSeq + val completedJobs = listener.completedJobs.reverse + val failedJobs = listener.failedJobs.reverse val activeJobsTable = jobsTable(request, "active", "activeJob", activeJobs, killEnabled = parent.killEnabled) diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index e5d408a16736..f4786e3931c9 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -473,7 +473,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() eventually(timeout(5 seconds), interval(50 milliseconds)) { val url = new URL( - sc.ui.get.appUIAddress.stripSuffix("/") + "/stages/stage/kill/?id=0") + sc.ui.get.webUrl.stripSuffix("/") + "/stages/stage/kill/?id=0") // SPARK-6846: should be POST only but YARN AM doesn't proxy POST getResponseCode(url, "GET") should be (200) getResponseCode(url, "POST") should be (200) @@ -486,7 +486,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() eventually(timeout(5 seconds), interval(50 milliseconds)) { val url = new URL( - sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/kill/?id=0") + sc.ui.get.webUrl.stripSuffix("/") + "/jobs/job/kill/?id=0") // SPARK-6846: should be POST only but YARN AM doesn't proxy POST getResponseCode(url, "GET") should be (200) getResponseCode(url, "POST") should be (200) @@ -620,7 +620,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B test("live UI json application list") { withSpark(newSparkContext()) { sc => val appListRawJson = HistoryServerSuite.getUrl(new URL( - sc.ui.get.appUIAddress + "/api/v1/applications")) + sc.ui.get.webUrl + "/api/v1/applications")) val appListJsonAst = JsonMethods.parse(appListRawJson) appListJsonAst.children.length should be (1) val attempts = (appListJsonAst \ "attempts").children @@ -640,7 +640,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity) rdd.count() - val stage0 = Source.fromURL(sc.ui.get.appUIAddress + + val stage0 = Source.fromURL(sc.ui.get.webUrl + "/stages/stage/?id=0&attempt=0&expandDagViz=true").mkString assert(stage0.contains("digraph G {\n subgraph clusterstage_0 {\n " + "label="Stage 0";\n subgraph ")) @@ -651,7 +651,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B assert(stage0.contains("{\n label="groupBy";\n " + "2 [label="MapPartitionsRDD [2]")) - val stage1 = Source.fromURL(sc.ui.get.appUIAddress + + val stage1 = Source.fromURL(sc.ui.get.webUrl + "/stages/stage/?id=1&attempt=0&expandDagViz=true").mkString assert(stage1.contains("digraph G {\n subgraph clusterstage_1 {\n " + "label="Stage 1";\n subgraph ")) @@ -662,7 +662,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B assert(stage1.contains("{\n label="groupBy";\n " + "5 [label="MapPartitionsRDD [5]")) - val stage2 = Source.fromURL(sc.ui.get.appUIAddress + + val stage2 = Source.fromURL(sc.ui.get.webUrl + "/stages/stage/?id=2&attempt=0&expandDagViz=true").mkString assert(stage2.contains("digraph G {\n subgraph clusterstage_2 {\n " + "label="Stage 2";\n subgraph ")) @@ -687,7 +687,7 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B } def goToUi(ui: SparkUI, path: String): Unit = { - go to (ui.appUIAddress.stripSuffix("/") + path) + go to (ui.webUrl.stripSuffix("/") + path) } def parseDate(json: JValue): Long = { @@ -699,6 +699,6 @@ class UISeleniumSuite extends SparkFunSuite with WebBrowser with Matchers with B } def apiUrl(ui: SparkUI, path: String): URL = { - new URL(ui.appUIAddress + "/api/v1/applications/" + ui.sc.get.applicationId + "/" + path) + new URL(ui.webUrl + "/api/v1/applications/" + ui.sc.get.applicationId + "/" + path) } } diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 4abcfb7e5191..68c7657cb315 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -66,7 +66,7 @@ class UISuite extends SparkFunSuite { withSpark(newSparkContext()) { sc => // test if the ui is visible, and all the expected tabs are visible eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(sc.ui.get.appUIAddress).mkString + val html = Source.fromURL(sc.ui.get.webUrl).mkString assert(!html.contains("random data that should not be present")) assert(html.toLowerCase.contains("stages")) assert(html.toLowerCase.contains("storage")) @@ -176,19 +176,18 @@ class UISuite extends SparkFunSuite { } } - test("verify appUIAddress contains the scheme") { + test("verify webUrl contains the scheme") { withSpark(newSparkContext()) { sc => val ui = sc.ui.get - val uiAddress = ui.appUIAddress - val uiHostPort = ui.appUIHostPort - assert(uiAddress.equals("http://" + uiHostPort)) + val uiAddress = ui.webUrl + assert(uiAddress.startsWith("http://") || uiAddress.startsWith("https://")) } } - test("verify appUIAddress contains the port") { + test("verify webUrl contains the port") { withSpark(newSparkContext()) { sc => val ui = sc.ui.get - val splitUIAddress = ui.appUIAddress.split(':') + val splitUIAddress = ui.webUrl.split(':') val boundPort = ui.boundPort assert(splitUIAddress(2).toInt == boundPort) } diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 5063c1fe988b..842c05e7bf73 100644 --- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -158,7 +158,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( sc.sparkUser, sc.appName, sc.conf, - sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.appUIAddress)), + sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.webUrl)), None, None, sc.conf.getOption("spark.mesos.driver.frameworkId") diff --git a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala index 09a252f3c74a..c1aa00151e69 100644 --- a/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala +++ b/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala @@ -77,7 +77,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( sc.sparkUser, sc.appName, sc.conf, - sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.appUIAddress)), + sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.webUrl)), Option.empty, Option.empty, sc.conf.getOption("spark.mesos.driver.frameworkId") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index 454c3dffa3db..e7cec999c219 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -92,13 +92,13 @@ class UISeleniumSuite val sparkUI = ssc.sparkContext.ui.get eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sparkUI.appUIAddress.stripSuffix("/")) + go to (sparkUI.webUrl.stripSuffix("/")) find(cssSelector( """ul li a[href*="streaming"]""")) should not be (None) } eventually(timeout(10 seconds), interval(50 milliseconds)) { // check whether streaming page exists - go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming") + go to (sparkUI.webUrl.stripSuffix("/") + "/streaming") val h3Text = findAll(cssSelector("h3")).map(_.text).toSeq h3Text should contain("Streaming Statistics") @@ -180,23 +180,23 @@ class UISeleniumSuite jobDetails should contain("Completed Stages:") // Check a batch page without id - go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming/batch/") + go to (sparkUI.webUrl.stripSuffix("/") + "/streaming/batch/") webDriver.getPageSource should include ("Missing id parameter") // Check a non-exist batch - go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming/batch/?id=12345") + go to (sparkUI.webUrl.stripSuffix("/") + "/streaming/batch/?id=12345") webDriver.getPageSource should include ("does not exist") } ssc.stop(false) eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sparkUI.appUIAddress.stripSuffix("/")) + go to (sparkUI.webUrl.stripSuffix("/")) find(cssSelector( """ul li a[href*="streaming"]""")) should be(None) } eventually(timeout(10 seconds), interval(50 milliseconds)) { - go to (sparkUI.appUIAddress.stripSuffix("/") + "/streaming") + go to (sparkUI.webUrl.stripSuffix("/") + "/streaming") val h3Text = findAll(cssSelector("h3")).map(_.text).toSeq h3Text should not contain("Streaming Statistics") } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index aabae140af8b..f2b9dfb4d184 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -406,7 +406,7 @@ private[spark] class ApplicationMaster( sc.getConf.get("spark.driver.host"), sc.getConf.get("spark.driver.port"), isClusterMode = true) - registerAM(sc.getConf, rpcEnv, driverRef, sc.ui.map(_.appUIAddress).getOrElse(""), + registerAM(sc.getConf, rpcEnv, driverRef, sc.ui.map(_.webUrl).getOrElse(""), securityMgr) } else { // Sanity check; should never happen in normal operation, since sc should only be null diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index d8b36c5feaf5..60da356ad14a 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -44,7 +44,7 @@ private[spark] class YarnClientSchedulerBackend( val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort - sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.appUIAddress) } + sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.webUrl) } val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ("--arg", hostport) From 9c8deef64efee20a0ddc9b612f90e77c80aede60 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 2 Nov 2016 09:39:15 +0000 Subject: [PATCH 046/132] [SPARK-18076][CORE][SQL] Fix default Locale used in DateFormat, NumberFormat to Locale.US ## What changes were proposed in this pull request? Fix `Locale.US` for all usages of `DateFormat`, `NumberFormat` ## How was this patch tested? Existing tests. Author: Sean Owen Closes #15610 from srowen/SPARK-18076. --- .../org/apache/spark/SparkHadoopWriter.scala | 8 +++---- .../apache/spark/deploy/SparkHadoopUtil.scala | 4 ++-- .../apache/spark/deploy/master/Master.scala | 5 ++-- .../apache/spark/deploy/worker/Worker.scala | 4 ++-- .../org/apache/spark/rdd/HadoopRDD.scala | 5 ++-- .../org/apache/spark/rdd/NewHadoopRDD.scala | 4 ++-- .../apache/spark/rdd/PairRDDFunctions.scala | 4 ++-- .../status/api/v1/JacksonMessageWriter.scala | 4 ++-- .../spark/status/api/v1/SimpleDateParam.scala | 6 ++--- .../scala/org/apache/spark/ui/UIUtils.scala | 3 ++- .../spark/util/logging/RollingPolicy.scala | 6 ++--- .../org/apache/spark/util/UtilsSuite.scala | 2 +- .../deploy/rest/mesos/MesosRestServer.scala | 11 ++++----- .../mllib/pmml/export/PMMLModelExport.scala | 4 ++-- .../expressions/datetimeExpressions.scala | 17 ++++++------- .../expressions/stringExpressions.scala | 2 +- .../spark/sql/catalyst/json/JSONOptions.scala | 6 +++-- .../sql/catalyst/util/DateTimeUtils.scala | 6 ++--- .../expressions/DateExpressionsSuite.scala | 24 +++++++++---------- .../catalyst/util/DateTimeUtilsSuite.scala | 6 ++--- .../datasources/csv/CSVInferSchema.scala | 4 ++-- .../datasources/csv/CSVOptions.scala | 5 ++-- .../sql/execution/metric/SQLMetrics.scala | 2 +- .../sql/execution/streaming/socket.scala | 4 ++-- .../apache/spark/sql/DateFunctionsSuite.scala | 11 +++++---- .../execution/datasources/csv/CSVSuite.scala | 9 +++---- .../datasources/csv/CSVTypeCastSuite.scala | 9 ++++--- .../hive/execution/InsertIntoHiveTable.scala | 9 +++---- .../spark/sql/hive/hiveWriterContainers.scala | 4 ++-- .../sql/sources/SimpleTextRelation.scala | 3 ++- .../apache/spark/streaming/ui/UIUtils.scala | 8 ++++--- 31 files changed, 103 insertions(+), 96 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 6550d703bc86..7f75a393bf8f 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -20,7 +20,7 @@ package org.apache.spark import java.io.IOException import java.text.NumberFormat import java.text.SimpleDateFormat -import java.util.Date +import java.util.{Date, Locale} import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path @@ -67,12 +67,12 @@ class SparkHadoopWriter(jobConf: JobConf) extends Logging with Serializable { def setup(jobid: Int, splitid: Int, attemptid: Int) { setIDs(jobid, splitid, attemptid) - HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmmss").format(now), + HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(now), jobid, splitID, attemptID, conf.value) } def open() { - val numfmt = NumberFormat.getInstance() + val numfmt = NumberFormat.getInstance(Locale.US) numfmt.setMinimumIntegerDigits(5) numfmt.setGroupingUsed(false) @@ -162,7 +162,7 @@ class SparkHadoopWriter(jobConf: JobConf) extends Logging with Serializable { private[spark] object SparkHadoopWriter { def createJobID(time: Date, id: Int): JobID = { - val formatter = new SimpleDateFormat("yyyyMMddHHmmss") + val formatter = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) val jobtrackerID = formatter.format(time) new JobID(jobtrackerID, id) } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 3f54ecc17ac3..23156072c3eb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -21,7 +21,7 @@ import java.io.IOException import java.lang.reflect.Method import java.security.PrivilegedExceptionAction import java.text.DateFormat -import java.util.{Arrays, Comparator, Date} +import java.util.{Arrays, Comparator, Date, Locale} import scala.collection.JavaConverters._ import scala.util.control.NonFatal @@ -357,7 +357,7 @@ class SparkHadoopUtil extends Logging { * @return a printable string value. */ private[spark] def tokenToString(token: Token[_ <: TokenIdentifier]): String = { - val df = DateFormat.getDateTimeInstance(DateFormat.SHORT, DateFormat.SHORT) + val df = DateFormat.getDateTimeInstance(DateFormat.SHORT, DateFormat.SHORT, Locale.US) val buffer = new StringBuilder(128) buffer.append(token.toString) try { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 8c91aa15167c..4618e6117a4f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.master import java.text.SimpleDateFormat -import java.util.Date +import java.util.{Date, Locale} import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} @@ -51,7 +51,8 @@ private[deploy] class Master( private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs + // For application IDs + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) private val WORKER_TIMEOUT_MS = conf.getLong("spark.worker.timeout", 60) * 1000 private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 0bedd9a20a96..8b1c6bf2e5fd 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.worker import java.io.File import java.io.IOException import java.text.SimpleDateFormat -import java.util.{Date, UUID} +import java.util.{Date, Locale, UUID} import java.util.concurrent._ import java.util.concurrent.{Future => JFuture, ScheduledFuture => JScheduledFuture} @@ -68,7 +68,7 @@ private[deploy] class Worker( ThreadUtils.newDaemonSingleThreadExecutor("worker-cleanup-thread")) // For worker and executor IDs - private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) // Send a heartbeat every (heartbeat timeout) / 4 milliseconds private val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index e1cf3938de09..36a2f5c87e37 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.rdd import java.io.IOException import java.text.SimpleDateFormat -import java.util.Date +import java.util.{Date, Locale} import scala.collection.immutable.Map import scala.reflect.ClassTag @@ -243,7 +243,8 @@ class HadoopRDD[K, V]( var reader: RecordReader[K, V] = null val inputFormat = getInputFormat(jobConf) - HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmmss").format(createTime), + HadoopRDD.addLocalConfiguration( + new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(createTime), context.stageId, theSplit.index, context.attemptNumber, jobConf) reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL) diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index baf31fb65887..488e777fea37 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.rdd import java.io.IOException import java.text.SimpleDateFormat -import java.util.Date +import java.util.{Date, Locale} import scala.reflect.ClassTag @@ -79,7 +79,7 @@ class NewHadoopRDD[K, V]( // private val serializableConf = new SerializableWritable(_conf) private val jobTrackerId: String = { - val formatter = new SimpleDateFormat("yyyyMMddHHmmss") + val formatter = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) formatter.format(new Date()) } diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 068f4ed8ad74..67baad1c51bc 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -19,7 +19,7 @@ package org.apache.spark.rdd import java.nio.ByteBuffer import java.text.SimpleDateFormat -import java.util.{Date, HashMap => JHashMap} +import java.util.{Date, HashMap => JHashMap, Locale} import scala.collection.{mutable, Map} import scala.collection.JavaConverters._ @@ -1079,7 +1079,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). val hadoopConf = conf val job = NewAPIHadoopJob.getInstance(hadoopConf) - val formatter = new SimpleDateFormat("yyyyMMddHHmmss") + val formatter = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) val jobtrackerID = formatter.format(new Date()) val stageId = self.id val jobConfiguration = job.getConfiguration diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala index f6a9f9c5573d..76af33c1a18d 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala @@ -21,7 +21,7 @@ import java.lang.annotation.Annotation import java.lang.reflect.Type import java.nio.charset.StandardCharsets import java.text.SimpleDateFormat -import java.util.{Calendar, SimpleTimeZone} +import java.util.{Calendar, Locale, SimpleTimeZone} import javax.ws.rs.Produces import javax.ws.rs.core.{MediaType, MultivaluedMap} import javax.ws.rs.ext.{MessageBodyWriter, Provider} @@ -86,7 +86,7 @@ private[v1] class JacksonMessageWriter extends MessageBodyWriter[Object]{ private[spark] object JacksonMessageWriter { def makeISODateFormat: SimpleDateFormat = { - val iso8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'GMT'") + val iso8601 = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'GMT'", Locale.US) val cal = Calendar.getInstance(new SimpleTimeZone(0, "GMT")) iso8601.setCalendar(cal) iso8601 diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala index 0c71cd238222..d8d5e8958b23 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/SimpleDateParam.scala @@ -17,7 +17,7 @@ package org.apache.spark.status.api.v1 import java.text.{ParseException, SimpleDateFormat} -import java.util.TimeZone +import java.util.{Locale, TimeZone} import javax.ws.rs.WebApplicationException import javax.ws.rs.core.Response import javax.ws.rs.core.Response.Status @@ -25,12 +25,12 @@ import javax.ws.rs.core.Response.Status private[v1] class SimpleDateParam(val originalValue: String) { val timestamp: Long = { - val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSz") + val format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSz", Locale.US) try { format.parse(originalValue).getTime() } catch { case _: ParseException => - val gmtDay = new SimpleDateFormat("yyyy-MM-dd") + val gmtDay = new SimpleDateFormat("yyyy-MM-dd", Locale.US) gmtDay.setTimeZone(TimeZone.getTimeZone("GMT")) try { gmtDay.parse(originalValue).getTime() diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index c0d1a2220f62..66b097aa8166 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -36,7 +36,8 @@ private[spark] object UIUtils extends Logging { // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. private val dateFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + override def initialValue(): SimpleDateFormat = + new SimpleDateFormat("yyyy/MM/dd HH:mm:ss", Locale.US) } def formatDate(date: Date): String = dateFormat.get.format(date) diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala index 5c4238c0381a..1f263df57c85 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingPolicy.scala @@ -18,7 +18,7 @@ package org.apache.spark.util.logging import java.text.SimpleDateFormat -import java.util.Calendar +import java.util.{Calendar, Locale} import org.apache.spark.internal.Logging @@ -59,7 +59,7 @@ private[spark] class TimeBasedRollingPolicy( } @volatile private var nextRolloverTime = calculateNextRolloverTime() - private val formatter = new SimpleDateFormat(rollingFileSuffixPattern) + private val formatter = new SimpleDateFormat(rollingFileSuffixPattern, Locale.US) /** Should rollover if current time has exceeded next rollover time */ def shouldRollover(bytesToBeWritten: Long): Boolean = { @@ -109,7 +109,7 @@ private[spark] class SizeBasedRollingPolicy( } @volatile private var bytesWrittenSinceRollover = 0L - val formatter = new SimpleDateFormat("--yyyy-MM-dd--HH-mm-ss--SSSS") + val formatter = new SimpleDateFormat("--yyyy-MM-dd--HH-mm-ss--SSSS", Locale.US) /** Should rollover if the next set of bytes is going to exceed the size limit */ def shouldRollover(bytesToBeWritten: Long): Boolean = { diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 15ef32f21d90..feacfb7642f2 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -264,7 +264,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { val hour = minute * 60 def str: (Long) => String = Utils.msDurationToString(_) - val sep = new DecimalFormatSymbols(Locale.getDefault()).getDecimalSeparator() + val sep = new DecimalFormatSymbols(Locale.US).getDecimalSeparator assert(str(123) === "123 ms") assert(str(second) === "1" + sep + "0 s") diff --git a/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala index 3b96488a129a..ff60b88c6d53 100644 --- a/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala +++ b/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.rest.mesos import java.io.File import java.text.SimpleDateFormat -import java.util.Date +import java.util.{Date, Locale} import java.util.concurrent.atomic.AtomicLong import javax.servlet.http.HttpServletResponse @@ -62,11 +62,10 @@ private[mesos] class MesosSubmitRequestServlet( private val DEFAULT_CORES = 1.0 private val nextDriverNumber = new AtomicLong(0) - private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs - private def newDriverId(submitDate: Date): String = { - "driver-%s-%04d".format( - createDateFormat.format(submitDate), nextDriverNumber.incrementAndGet()) - } + // For application IDs + private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) + private def newDriverId(submitDate: Date): String = + f"driver-${createDateFormat.format(submitDate)}-${nextDriverNumber.incrementAndGet()}%04d" /** * Build a driver description from the fields specified in the submit request. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala index 426bb818c926..f5ca1c221d66 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExport.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.pmml.export import java.text.SimpleDateFormat -import java.util.Date +import java.util.{Date, Locale} import scala.beans.BeanProperty @@ -34,7 +34,7 @@ private[mllib] trait PMMLModelExport { val version = getClass.getPackage.getImplementationVersion val app = new Application("Apache Spark MLlib").setVersion(version) val timestamp = new Timestamp() - .addContent(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss").format(new Date())) + .addContent(new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss", Locale.US).format(new Date())) val header = new Header() .setApplication(app) .setTimestamp(timestamp) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 7ab68a13e09c..67c078ae5e26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.text.SimpleDateFormat -import java.util.{Calendar, TimeZone} +import java.util.{Calendar, Locale, TimeZone} import scala.util.Try @@ -331,7 +331,7 @@ case class DateFormatClass(left: Expression, right: Expression) extends BinaryEx override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType) override protected def nullSafeEval(timestamp: Any, format: Any): Any = { - val sdf = new SimpleDateFormat(format.toString) + val sdf = new SimpleDateFormat(format.toString, Locale.US) UTF8String.fromString(sdf.format(new java.util.Date(timestamp.asInstanceOf[Long] / 1000))) } @@ -400,7 +400,7 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] private lazy val formatter: SimpleDateFormat = - Try(new SimpleDateFormat(constFormat.toString)).getOrElse(null) + Try(new SimpleDateFormat(constFormat.toString, Locale.US)).getOrElse(null) override def eval(input: InternalRow): Any = { val t = left.eval(input) @@ -425,7 +425,7 @@ abstract class UnixTime extends BinaryExpression with ExpectsInputTypes { null } else { val formatString = f.asInstanceOf[UTF8String].toString - Try(new SimpleDateFormat(formatString).parse( + Try(new SimpleDateFormat(formatString, Locale.US).parse( t.asInstanceOf[UTF8String].toString).getTime / 1000L).getOrElse(null) } } @@ -520,7 +520,7 @@ case class FromUnixTime(sec: Expression, format: Expression) private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] private lazy val formatter: SimpleDateFormat = - Try(new SimpleDateFormat(constFormat.toString)).getOrElse(null) + Try(new SimpleDateFormat(constFormat.toString, Locale.US)).getOrElse(null) override def eval(input: InternalRow): Any = { val time = left.eval(input) @@ -539,9 +539,10 @@ case class FromUnixTime(sec: Expression, format: Expression) if (f == null) { null } else { - Try(UTF8String.fromString(new SimpleDateFormat( - f.asInstanceOf[UTF8String].toString).format(new java.util.Date( - time.asInstanceOf[Long] * 1000L)))).getOrElse(null) + Try( + UTF8String.fromString(new SimpleDateFormat(f.toString, Locale.US). + format(new java.util.Date(time.asInstanceOf[Long] * 1000L))) + ).getOrElse(null) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index 1bcbb6cfc924..25a5e3fd7da7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -1415,7 +1415,7 @@ case class Sentences( val locale = if (languageStr != null && countryStr != null) { new Locale(languageStr.toString, countryStr.toString) } else { - Locale.getDefault + Locale.US } getSentences(string.asInstanceOf[UTF8String].toString, locale) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala index aec18922ea6c..c45970658cf0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.json +import java.util.Locale + import com.fasterxml.jackson.core.{JsonFactory, JsonParser} import org.apache.commons.lang3.time.FastDateFormat @@ -56,11 +58,11 @@ private[sql] class JSONOptions( // Uses `FastDateFormat` which can be direct replacement for `SimpleDateFormat` and thread-safe. val dateFormat: FastDateFormat = - FastDateFormat.getInstance(parameters.getOrElse("dateFormat", "yyyy-MM-dd")) + FastDateFormat.getInstance(parameters.getOrElse("dateFormat", "yyyy-MM-dd"), Locale.US) val timestampFormat: FastDateFormat = FastDateFormat.getInstance( - parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSZZ")) + parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSZZ"), Locale.US) // Parse mode flags if (!ParseModes.isValidMode(parseMode)) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 0b643a5b8426..235ca8d2633a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.text.{DateFormat, SimpleDateFormat} -import java.util.{Calendar, TimeZone} +import java.util.{Calendar, Locale, TimeZone} import javax.xml.bind.DatatypeConverter import scala.annotation.tailrec @@ -79,14 +79,14 @@ object DateTimeUtils { // `SimpleDateFormat` is not thread-safe. val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { override def initialValue(): SimpleDateFormat = { - new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) } } // `SimpleDateFormat` is not thread-safe. private val threadLocalDateFormat = new ThreadLocal[DateFormat] { override def initialValue(): SimpleDateFormat = { - new SimpleDateFormat("yyyy-MM-dd") + new SimpleDateFormat("yyyy-MM-dd", Locale.US) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 6118a34d29ea..35cea25ba0b7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.util.Calendar +import java.util.{Calendar, Locale} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -30,8 +30,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { import IntegralLiteralTestUtils._ - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val sdfDate = new SimpleDateFormat("yyyy-MM-dd") + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + val sdfDate = new SimpleDateFormat("yyyy-MM-dd", Locale.US) val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) @@ -49,7 +49,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("DayOfYear") { - val sdfDay = new SimpleDateFormat("D") + val sdfDay = new SimpleDateFormat("D", Locale.US) (0 to 3).foreach { m => (0 to 5).foreach { i => val c = Calendar.getInstance() @@ -411,9 +411,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("from_unixtime") { - val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" - val sdf2 = new SimpleDateFormat(fmt2) + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) checkEvaluation( FromUnixTime(Literal(0L), Literal("yyyy-MM-dd HH:mm:ss")), sdf1.format(new Timestamp(0))) checkEvaluation(FromUnixTime( @@ -430,11 +430,11 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("unix_timestamp") { - val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" - val sdf2 = new SimpleDateFormat(fmt2) + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) val fmt3 = "yy-MM-dd" - val sdf3 = new SimpleDateFormat(fmt3) + val sdf3 = new SimpleDateFormat(fmt3, Locale.US) val date1 = Date.valueOf("2015-07-24") checkEvaluation( UnixTimestamp(Literal(sdf1.format(new Timestamp(0))), Literal("yyyy-MM-dd HH:mm:ss")), 0L) @@ -466,11 +466,11 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("to_unix_timestamp") { - val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" - val sdf2 = new SimpleDateFormat(fmt2) + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) val fmt3 = "yy-MM-dd" - val sdf3 = new SimpleDateFormat(fmt3) + val sdf3 = new SimpleDateFormat(fmt3, Locale.US) val date1 = Date.valueOf("2015-07-24") checkEvaluation( ToUnixTimestamp(Literal(sdf1.format(new Timestamp(0))), Literal("yyyy-MM-dd HH:mm:ss")), 0L) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 4f516d006458..e0a9a0c3d5c0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.util.{Calendar, TimeZone} +import java.util.{Calendar, Locale, TimeZone} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.DateTimeUtils._ @@ -68,8 +68,8 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(d2.toString === d1.toString) } - val df1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val df2 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss z") + val df1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + val df2 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss z", Locale.US) checkFromToJavaDate(new Date(100)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala index 3ab775c90923..1981d8607c0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala @@ -247,7 +247,7 @@ private[csv] object CSVTypeCast { case options.positiveInf => Float.PositiveInfinity case _ => Try(datum.toFloat) - .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).floatValue()) + .getOrElse(NumberFormat.getInstance(Locale.US).parse(datum).floatValue()) } case _: DoubleType => datum match { @@ -256,7 +256,7 @@ private[csv] object CSVTypeCast { case options.positiveInf => Double.PositiveInfinity case _ => Try(datum.toDouble) - .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).doubleValue()) + .getOrElse(NumberFormat.getInstance(Locale.US).parse(datum).doubleValue()) } case _: BooleanType => datum.toBoolean case dt: DecimalType => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala index 014614eb997a..5903729c11fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources.csv import java.nio.charset.StandardCharsets +import java.util.Locale import org.apache.commons.lang3.time.FastDateFormat @@ -104,11 +105,11 @@ private[csv] class CSVOptions(@transient private val parameters: Map[String, Str // Uses `FastDateFormat` which can be direct replacement for `SimpleDateFormat` and thread-safe. val dateFormat: FastDateFormat = - FastDateFormat.getInstance(parameters.getOrElse("dateFormat", "yyyy-MM-dd")) + FastDateFormat.getInstance(parameters.getOrElse("dateFormat", "yyyy-MM-dd"), Locale.US) val timestampFormat: FastDateFormat = FastDateFormat.getInstance( - parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSZZ")) + parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSZZ"), Locale.US) val maxColumns = getInt("maxColumns", 20480) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala index 0cc1edd196bc..dbc27d8b237f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLMetrics.scala @@ -102,7 +102,7 @@ object SQLMetrics { */ def stringValue(metricsType: String, values: Seq[Long]): String = { if (metricsType == SUM_METRIC) { - val numberFormat = NumberFormat.getIntegerInstance(Locale.ENGLISH) + val numberFormat = NumberFormat.getIntegerInstance(Locale.US) numberFormat.format(values.sum) } else { val strFormat: Long => String = if (metricsType == SIZE_METRIC) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala index c662e7c6bc77..042977f870b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/socket.scala @@ -21,7 +21,7 @@ import java.io.{BufferedReader, InputStreamReader, IOException} import java.net.Socket import java.sql.Timestamp import java.text.SimpleDateFormat -import java.util.Calendar +import java.util.{Calendar, Locale} import javax.annotation.concurrent.GuardedBy import scala.collection.mutable.ListBuffer @@ -37,7 +37,7 @@ object TextSocketSource { val SCHEMA_REGULAR = StructType(StructField("value", StringType) :: Nil) val SCHEMA_TIMESTAMP = StructType(StructField("value", StringType) :: StructField("timestamp", TimestampType) :: Nil) - val DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index f7aa3b747ae5..e05b2252ee34 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat +import java.util.Locale import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.functions._ @@ -55,8 +56,8 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { checkAnswer(sql("""SELECT CURRENT_TIMESTAMP() = NOW()"""), Row(true)) } - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") - val sdfDate = new SimpleDateFormat("yyyy-MM-dd") + val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + val sdfDate = new SimpleDateFormat("yyyy-MM-dd", Locale.US) val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) val ts = new Timestamp(sdf.parse("2013-04-08 13:10:15").getTime) @@ -395,11 +396,11 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { } test("from_unixtime") { - val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" - val sdf2 = new SimpleDateFormat(fmt2) + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) val fmt3 = "yy-MM-dd HH-mm-ss" - val sdf3 = new SimpleDateFormat(fmt3) + val sdf3 = new SimpleDateFormat(fmt3, Locale.US) val df = Seq((1000, "yyyy-MM-dd HH:mm:ss.SSS"), (-1000, "yy-MM-dd HH-mm-ss")).toDF("a", "b") checkAnswer( df.select(from_unixtime(col("a"))), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index f7c22c6c93f7..8209b5bd7f9d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -21,6 +21,7 @@ import java.io.File import java.nio.charset.UnsupportedCharsetException import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat +import java.util.Locale import org.apache.commons.lang3.time.FastDateFormat import org.apache.hadoop.io.SequenceFile.CompressionType @@ -487,7 +488,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { .select("date") .collect() - val dateFormat = new SimpleDateFormat("dd/MM/yyyy HH:mm") + val dateFormat = new SimpleDateFormat("dd/MM/yyyy HH:mm", Locale.US) val expected = Seq(Seq(new Timestamp(dateFormat.parse("26/08/2015 18:00").getTime)), Seq(new Timestamp(dateFormat.parse("27/10/2014 18:30").getTime)), @@ -509,7 +510,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { .select("date") .collect() - val dateFormat = new SimpleDateFormat("dd/MM/yyyy hh:mm") + val dateFormat = new SimpleDateFormat("dd/MM/yyyy hh:mm", Locale.US) val expected = Seq( new Date(dateFormat.parse("26/08/2015 18:00").getTime), new Date(dateFormat.parse("27/10/2014 18:30").getTime), @@ -728,7 +729,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { .option("inferSchema", "false") .load(iso8601timestampsPath) - val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ") + val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ", Locale.US) val expectedTimestamps = timestamps.collect().map { r => // This should be ISO8601 formatted string. Row(iso8501.format(r.toSeq.head)) @@ -761,7 +762,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { .option("inferSchema", "false") .load(iso8601datesPath) - val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd") + val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd", Locale.US) val expectedDates = dates.collect().map { r => // This should be ISO8601 formatted string. Row(iso8501.format(r.toSeq.head)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala index 51832a13cfe0..c74406b9cbfb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala @@ -144,13 +144,12 @@ class CSVTypeCastSuite extends SparkFunSuite { DateTimeUtils.millisToDays(DateTimeUtils.stringToTime("2015-01-01").getTime)) } - test("Float and Double Types are cast correctly with Locale") { + test("Float and Double Types are cast without respect to platform default Locale") { val originalLocale = Locale.getDefault try { - val locale : Locale = new Locale("fr", "FR") - Locale.setDefault(locale) - assert(CSVTypeCast.castTo("1,00", FloatType) == 1.0) - assert(CSVTypeCast.castTo("1,00", DoubleType) == 1.0) + Locale.setDefault(new Locale("fr", "FR")) + assert(CSVTypeCast.castTo("1,00", FloatType) == 100.0) // Would parse as 1.0 in fr-FR + assert(CSVTypeCast.castTo("1,00", DoubleType) == 100.0) } finally { Locale.setDefault(originalLocale) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 2843100fb3b3..05164d774cca 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -20,9 +20,7 @@ package org.apache.spark.sql.hive.execution import java.io.IOException import java.net.URI import java.text.SimpleDateFormat -import java.util.{Date, Random} - -import scala.collection.JavaConverters._ +import java.util.{Date, Locale, Random} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} @@ -60,9 +58,8 @@ case class InsertIntoHiveTable( private def executionId: String = { val rand: Random = new Random - val format: SimpleDateFormat = new SimpleDateFormat("yyyy-MM-dd_HH-mm-ss_SSS") - val executionId: String = "hive_" + format.format(new Date) + "_" + Math.abs(rand.nextLong) - return executionId + val format = new SimpleDateFormat("yyyy-MM-dd_HH-mm-ss_SSS", Locale.US) + "hive_" + format.format(new Date) + "_" + Math.abs(rand.nextLong) } private def getStagingDir(inputPath: Path, hadoopConf: Configuration): Path = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index ea88276bb96c..e53c3e4d4833 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive import java.text.NumberFormat -import java.util.Date +import java.util.{Date, Locale} import scala.collection.JavaConverters._ @@ -95,7 +95,7 @@ private[hive] class SparkHiveWriterContainer( } protected def getOutputName: String = { - val numberFormat = NumberFormat.getInstance() + val numberFormat = NumberFormat.getInstance(Locale.US) numberFormat.setMinimumIntegerDigits(5) numberFormat.setGroupingUsed(false) val extension = Utilities.getFileExtension(conf.value, fileSinkConf.getCompressed, outputFormat) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala index 64d0ecbeefc9..cecfd9909865 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/SimpleTextRelation.scala @@ -18,6 +18,7 @@ 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} @@ -141,7 +142,7 @@ class SimpleTextOutputWriter(path: String, context: TaskAttemptContext) class AppendingTextOutputFormat(path: String) extends TextOutputFormat[NullWritable, Text] { - val numberFormat = NumberFormat.getInstance() + val numberFormat = NumberFormat.getInstance(Locale.US) numberFormat.setMinimumIntegerDigits(5) numberFormat.setGroupingUsed(false) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala index 9b1c939e9329..84ecf81abfbf 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/UIUtils.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.ui import java.text.SimpleDateFormat -import java.util.TimeZone +import java.util.{Locale, TimeZone} import java.util.concurrent.TimeUnit import scala.xml.Node @@ -80,11 +80,13 @@ private[streaming] object UIUtils { // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. private val batchTimeFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + override def initialValue(): SimpleDateFormat = + new SimpleDateFormat("yyyy/MM/dd HH:mm:ss", Locale.US) } private val batchTimeFormatWithMilliseconds = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS") + override def initialValue(): SimpleDateFormat = + new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS", Locale.US) } /** From f151bd1af8a05d4b6c901ebe6ac0b51a4a1a20df Mon Sep 17 00:00:00 2001 From: eyal farago Date: Wed, 2 Nov 2016 11:12:20 +0100 Subject: [PATCH 047/132] [SPARK-16839][SQL] Simplify Struct creation code path ## What changes were proposed in this pull request? Simplify struct creation, especially the aspect of `CleanupAliases` which missed some aliases when handling trees created by `CreateStruct`. This PR includes: 1. A failing test (create struct with nested aliases, some of the aliases survive `CleanupAliases`). 2. A fix that transforms `CreateStruct` into a `CreateNamedStruct` constructor, effectively eliminating `CreateStruct` from all expression trees. 3. A `NamePlaceHolder` used by `CreateStruct` when column names cannot be extracted from unresolved `NamedExpression`. 4. A new Analyzer rule that resolves `NamePlaceHolder` into a string literal once the `NamedExpression` is resolved. 5. `CleanupAliases` code was simplified as it no longer has to deal with `CreateStruct`'s top level columns. ## How was this patch tested? Running all tests-suits in package org.apache.spark.sql, especially including the analysis suite, making sure added test initially fails, after applying suggested fix rerun the entire analysis package successfully. Modified few tests that expected `CreateStruct` which is now transformed into `CreateNamedStruct`. Author: eyal farago Author: Herman van Hovell Author: eyal farago Author: Eyal Farago Author: Hyukjin Kwon Author: eyalfa Closes #15718 from hvanhovell/SPARK-16839-2. --- R/pkg/inst/tests/testthat/test_sparkSQL.R | 12 +- .../sql/catalyst/analysis/Analyzer.scala | 53 ++--- .../catalyst/analysis/FunctionRegistry.scala | 2 +- .../sql/catalyst/expressions/Projection.scala | 2 - .../expressions/complexTypeCreator.scala | 212 ++++++------------ .../sql/catalyst/parser/AstBuilder.scala | 4 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 38 +++- .../expressions/ComplexTypeSuite.scala | 1 - .../scala/org/apache/spark/sql/Column.scala | 3 + .../command/AnalyzeColumnCommand.scala | 4 +- .../sql-tests/results/group-by.sql.out | 2 +- .../apache/spark/sql/hive/test/TestHive.scala | 20 +- .../resources/sqlgen/subquery_in_having_2.sql | 2 +- .../sql/catalyst/LogicalPlanToSQLSuite.scala | 12 +- 14 files changed, 169 insertions(+), 198 deletions(-) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 806019d7524f..d7fe6b32822a 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1222,16 +1222,16 @@ test_that("column functions", { # Test struct() df <- createDataFrame(list(list(1L, 2L, 3L), list(4L, 5L, 6L)), schema = c("a", "b", "c")) - result <- collect(select(df, struct("a", "c"))) + result <- collect(select(df, alias(struct("a", "c"), "d"))) expected <- data.frame(row.names = 1:2) - expected$"struct(a, c)" <- list(listToStruct(list(a = 1L, c = 3L)), - listToStruct(list(a = 4L, c = 6L))) + expected$"d" <- list(listToStruct(list(a = 1L, c = 3L)), + listToStruct(list(a = 4L, c = 6L))) expect_equal(result, expected) - result <- collect(select(df, struct(df$a, df$b))) + result <- collect(select(df, alias(struct(df$a, df$b), "d"))) expected <- data.frame(row.names = 1:2) - expected$"struct(a, b)" <- list(listToStruct(list(a = 1L, b = 2L)), - listToStruct(list(a = 4L, b = 5L))) + expected$"d" <- list(listToStruct(list(a = 1L, b = 2L)), + listToStruct(list(a = 4L, b = 5L))) expect_equal(result, expected) # Test encode(), decode() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index f8f4799322b3..5011f2fdbf9b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, _} import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.catalyst.trees.{TreeNodeRef} +import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.toPrettySQL import org.apache.spark.sql.types._ @@ -83,6 +83,7 @@ class Analyzer( ResolveTableValuedFunctions :: ResolveRelations :: ResolveReferences :: + ResolveCreateNamedStruct :: ResolveDeserializer :: ResolveNewInstance :: ResolveUpCast :: @@ -653,11 +654,12 @@ class Analyzer( case s: Star => s.expand(child, resolver) case o => o :: Nil }) - case c: CreateStruct if containsStar(c.children) => - c.copy(children = c.children.flatMap { - case s: Star => s.expand(child, resolver) - case o => o :: Nil - }) + case c: CreateNamedStruct if containsStar(c.valExprs) => + val newChildren = c.children.grouped(2).flatMap { + case Seq(k, s : Star) => CreateStruct(s.expand(child, resolver)).children + case kv => kv + } + c.copy(children = newChildren.toList ) case c: CreateArray if containsStar(c.children) => c.copy(children = c.children.flatMap { case s: Star => s.expand(child, resolver) @@ -1141,7 +1143,7 @@ class Analyzer( case In(e, Seq(l @ ListQuery(_, exprId))) if e.resolved => // Get the left hand side expressions. val expressions = e match { - case CreateStruct(exprs) => exprs + case cns : CreateNamedStruct => cns.valExprs case expr => Seq(expr) } resolveSubQuery(l, plans, expressions.size) { (rewrite, conditions) => @@ -2072,18 +2074,8 @@ object EliminateUnions extends Rule[LogicalPlan] { */ object CleanupAliases extends Rule[LogicalPlan] { private def trimAliases(e: Expression): Expression = { - var stop = false e.transformDown { - // CreateStruct is a special case, we need to retain its top level Aliases as they decide the - // name of StructField. We also need to stop transform down this expression, or the Aliases - // under CreateStruct will be mistakenly trimmed. - case c: CreateStruct if !stop => - stop = true - c.copy(children = c.children.map(trimNonTopLevelAliases)) - case c: CreateStructUnsafe if !stop => - stop = true - c.copy(children = c.children.map(trimNonTopLevelAliases)) - case Alias(child, _) if !stop => child + case Alias(child, _) => child } } @@ -2116,15 +2108,8 @@ object CleanupAliases extends Rule[LogicalPlan] { case a: AppendColumns => a case other => - var stop = false other transformExpressionsDown { - case c: CreateStruct if !stop => - stop = true - c.copy(children = c.children.map(trimNonTopLevelAliases)) - case c: CreateStructUnsafe if !stop => - stop = true - c.copy(children = c.children.map(trimNonTopLevelAliases)) - case Alias(child, _) if !stop => child + case Alias(child, _) => child } } } @@ -2217,3 +2202,19 @@ object TimeWindowing extends Rule[LogicalPlan] { } } } + +/** + * Resolve a [[CreateNamedStruct]] if it contains [[NamePlaceholder]]s. + */ +object ResolveCreateNamedStruct extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan.transformAllExpressions { + case e: CreateNamedStruct if !e.resolved => + val children = e.children.grouped(2).flatMap { + case Seq(NamePlaceholder, e: NamedExpression) if e.resolved => + Seq(Literal(e.name), e) + case kv => + kv + } + CreateNamedStruct(children.toList) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 3e836ca375e2..b028d07fb8d0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -357,7 +357,7 @@ object FunctionRegistry { expression[MapValues]("map_values"), expression[Size]("size"), expression[SortArray]("sort_array"), - expression[CreateStruct]("struct"), + CreateStruct.registryEntry, // misc functions expression[AssertTrue]("assert_true"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index a81fa1ce3adc..03e054d09851 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -119,7 +119,6 @@ object UnsafeProjection { */ def create(exprs: Seq[Expression]): UnsafeProjection = { val unsafeExprs = exprs.map(_ transform { - case CreateStruct(children) => CreateStructUnsafe(children) case CreateNamedStruct(children) => CreateNamedStructUnsafe(children) }) GenerateUnsafeProjection.generate(unsafeExprs) @@ -145,7 +144,6 @@ object UnsafeProjection { subexpressionEliminationEnabled: Boolean): UnsafeProjection = { val e = exprs.map(BindReferences.bindReference(_, inputSchema)) .map(_ transform { - case CreateStruct(children) => CreateStructUnsafe(children) case CreateNamedStruct(children) => CreateNamedStructUnsafe(children) }) GenerateUnsafeProjection.generate(e, subexpressionEliminationEnabled) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index 917aa0873130..dbfb2996ec9d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -18,9 +18,11 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.analysis.Star import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData, TypeUtils} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, TypeUtils} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -172,101 +174,71 @@ case class CreateMap(children: Seq[Expression]) extends Expression { } /** - * Returns a Row containing the evaluation of all children expressions. + * An expression representing a not yet available attribute name. This expression is unevaluable + * and as its name suggests it is a temporary place holder until we're able to determine the + * actual attribute name. */ -@ExpressionDescription( - usage = "_FUNC_(col1, col2, col3, ...) - Creates a struct with the given field values.") -case class CreateStruct(children: Seq[Expression]) extends Expression { - - override def foldable: Boolean = children.forall(_.foldable) - - override lazy val dataType: StructType = { - val fields = children.zipWithIndex.map { case (child, idx) => - child match { - case ne: NamedExpression => - StructField(ne.name, ne.dataType, ne.nullable, ne.metadata) - case _ => - StructField(s"col${idx + 1}", child.dataType, child.nullable, Metadata.empty) - } - } - StructType(fields) - } - +case object NamePlaceholder extends LeafExpression with Unevaluable { + override lazy val resolved: Boolean = false + override def foldable: Boolean = false override def nullable: Boolean = false + override def dataType: DataType = StringType + override def prettyName: String = "NamePlaceholder" + override def toString: String = prettyName +} - override def eval(input: InternalRow): Any = { - InternalRow(children.map(_.eval(input)): _*) +/** + * Returns a Row containing the evaluation of all children expressions. + */ +object CreateStruct extends FunctionBuilder { + def apply(children: Seq[Expression]): CreateNamedStruct = { + CreateNamedStruct(children.zipWithIndex.flatMap { + case (e: NamedExpression, _) if e.resolved => Seq(Literal(e.name), e) + case (e: NamedExpression, _) => Seq(NamePlaceholder, e) + case (e, index) => Seq(Literal(s"col${index + 1}"), e) + }) } - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val rowClass = classOf[GenericInternalRow].getName - val values = ctx.freshName("values") - ctx.addMutableState("Object[]", values, s"this.$values = null;") - - ev.copy(code = s""" - boolean ${ev.isNull} = false; - this.$values = new Object[${children.size}];""" + - ctx.splitExpressions( - ctx.INPUT_ROW, - children.zipWithIndex.map { case (e, i) => - val eval = e.genCode(ctx) - eval.code + s""" - if (${eval.isNull}) { - $values[$i] = null; - } else { - $values[$i] = ${eval.value}; - }""" - }) + - s""" - final InternalRow ${ev.value} = new $rowClass($values); - this.$values = null; - """) + /** + * Entry to use in the function registry. + */ + val registryEntry: (String, (ExpressionInfo, FunctionBuilder)) = { + val info: ExpressionInfo = new ExpressionInfo( + "org.apache.spark.sql.catalyst.expressions.NamedStruct", + null, + "struct", + "_FUNC_(col1, col2, col3, ...) - Creates a struct with the given field values.", + "") + ("struct", (info, this)) } - - override def prettyName: String = "struct" } - /** - * Creates a struct with the given field names and values - * - * @param children Seq(name1, val1, name2, val2, ...) + * Common base class for both [[CreateNamedStruct]] and [[CreateNamedStructUnsafe]]. */ -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values.") -// scalastyle:on line.size.limit -case class CreateNamedStruct(children: Seq[Expression]) extends Expression { +trait CreateNamedStructLike extends Expression { + lazy val (nameExprs, valExprs) = children.grouped(2).map { + case Seq(name, value) => (name, value) + }.toList.unzip - /** - * Returns Aliased [[Expression]]s that could be used to construct a flattened version of this - * StructType. - */ - def flatten: Seq[NamedExpression] = valExprs.zip(names).map { - case (v, n) => Alias(v, n.toString)() - } + lazy val names = nameExprs.map(_.eval(EmptyRow)) - private lazy val (nameExprs, valExprs) = - children.grouped(2).map { case Seq(name, value) => (name, value) }.toList.unzip + override def nullable: Boolean = false - private lazy val names = nameExprs.map(_.eval(EmptyRow)) + override def foldable: Boolean = valExprs.forall(_.foldable) override lazy val dataType: StructType = { val fields = names.zip(valExprs).map { - case (name, valExpr: NamedExpression) => - StructField(name.asInstanceOf[UTF8String].toString, - valExpr.dataType, valExpr.nullable, valExpr.metadata) - case (name, valExpr) => - StructField(name.asInstanceOf[UTF8String].toString, - valExpr.dataType, valExpr.nullable, Metadata.empty) + case (name, expr) => + val metadata = expr match { + case ne: NamedExpression => ne.metadata + case _ => Metadata.empty + } + StructField(name.toString, expr.dataType, expr.nullable, metadata) } StructType(fields) } - override def foldable: Boolean = valExprs.forall(_.foldable) - - override def nullable: Boolean = false - override def checkInputDataTypes(): TypeCheckResult = { if (children.size % 2 != 0) { TypeCheckResult.TypeCheckFailure(s"$prettyName expects an even number of arguments.") @@ -274,8 +246,8 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression { val invalidNames = nameExprs.filterNot(e => e.foldable && e.dataType == StringType) if (invalidNames.nonEmpty) { TypeCheckResult.TypeCheckFailure( - s"Only foldable StringType expressions are allowed to appear at odd position , got :" + - s" ${invalidNames.mkString(",")}") + "Only foldable StringType expressions are allowed to appear at odd position, got:" + + s" ${invalidNames.mkString(",")}") } else if (!names.contains(null)) { TypeCheckResult.TypeCheckSuccess } else { @@ -284,9 +256,29 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression { } } + /** + * Returns Aliased [[Expression]]s that could be used to construct a flattened version of this + * StructType. + */ + def flatten: Seq[NamedExpression] = valExprs.zip(names).map { + case (v, n) => Alias(v, n.toString)() + } + override def eval(input: InternalRow): Any = { InternalRow(valExprs.map(_.eval(input)): _*) } +} + +/** + * Creates a struct with the given field names and values + * + * @param children Seq(name1, val1, name2, val2, ...) + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values.") +// scalastyle:on line.size.limit +case class CreateNamedStruct(children: Seq[Expression]) extends CreateNamedStructLike { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val rowClass = classOf[GenericInternalRow].getName @@ -316,44 +308,6 @@ case class CreateNamedStruct(children: Seq[Expression]) extends Expression { override def prettyName: String = "named_struct" } -/** - * Returns a Row containing the evaluation of all children expressions. This is a variant that - * returns UnsafeRow directly. The unsafe projection operator replaces [[CreateStruct]] with - * this expression automatically at runtime. - */ -case class CreateStructUnsafe(children: Seq[Expression]) extends Expression { - - override def foldable: Boolean = children.forall(_.foldable) - - override lazy val resolved: Boolean = childrenResolved - - override lazy val dataType: StructType = { - val fields = children.zipWithIndex.map { case (child, idx) => - child match { - case ne: NamedExpression => - StructField(ne.name, ne.dataType, ne.nullable, ne.metadata) - case _ => - StructField(s"col${idx + 1}", child.dataType, child.nullable, Metadata.empty) - } - } - StructType(fields) - } - - override def nullable: Boolean = false - - override def eval(input: InternalRow): Any = { - InternalRow(children.map(_.eval(input)): _*) - } - - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val eval = GenerateUnsafeProjection.createCode(ctx, children) - ExprCode(code = eval.code, isNull = eval.isNull, value = eval.value) - } - - override def prettyName: String = "struct_unsafe" -} - - /** * Creates a struct with the given field names and values. This is a variant that returns * UnsafeRow directly. The unsafe projection operator replaces [[CreateStruct]] with @@ -361,31 +315,7 @@ case class CreateStructUnsafe(children: Seq[Expression]) extends Expression { * * @param children Seq(name1, val1, name2, val2, ...) */ -case class CreateNamedStructUnsafe(children: Seq[Expression]) extends Expression { - - private lazy val (nameExprs, valExprs) = - children.grouped(2).map { case Seq(name, value) => (name, value) }.toList.unzip - - private lazy val names = nameExprs.map(_.eval(EmptyRow).toString) - - override lazy val dataType: StructType = { - val fields = names.zip(valExprs).map { - case (name, valExpr: NamedExpression) => - StructField(name, valExpr.dataType, valExpr.nullable, valExpr.metadata) - case (name, valExpr) => - StructField(name, valExpr.dataType, valExpr.nullable, Metadata.empty) - } - StructType(fields) - } - - override def foldable: Boolean = valExprs.forall(_.foldable) - - override def nullable: Boolean = false - - override def eval(input: InternalRow): Any = { - InternalRow(valExprs.map(_.eval(input)): _*) - } - +case class CreateNamedStructUnsafe(children: Seq[Expression]) extends CreateNamedStructLike { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val eval = GenerateUnsafeProjection.createCode(ctx, valExprs) ExprCode(code = eval.code, isNull = eval.isNull, value = eval.value) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index ac1577b3abb4..4b151c81d8f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -688,8 +688,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { // inline table comes in two styles: // style 1: values (1), (2), (3) -- multiple columns are supported // style 2: values 1, 2, 3 -- only a single column is supported here - case CreateStruct(children) => children // style 1 - case child => Seq(child) // style 2 + case struct: CreateNamedStruct => struct.valExprs // style 1 + case child => Seq(child) // style 2 } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 590774c04304..817de48de279 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.analysis +import org.scalatest.ShouldMatchers + import org.apache.spark.sql.catalyst.{SimpleCatalystConf, TableIdentifier} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -25,7 +27,8 @@ import org.apache.spark.sql.catalyst.plans.{Cross, Inner} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types._ -class AnalysisSuite extends AnalysisTest { + +class AnalysisSuite extends AnalysisTest with ShouldMatchers { import org.apache.spark.sql.catalyst.analysis.TestRelations._ test("union project *") { @@ -218,9 +221,36 @@ class AnalysisSuite extends AnalysisTest { // CreateStruct is a special case that we should not trim Alias for it. plan = testRelation.select(CreateStruct(Seq(a, (a + 1).as("a+1"))).as("col")) - checkAnalysis(plan, plan) - plan = testRelation.select(CreateStructUnsafe(Seq(a, (a + 1).as("a+1"))).as("col")) - checkAnalysis(plan, plan) + expected = testRelation.select(CreateNamedStruct(Seq( + Literal(a.name), a, + Literal("a+1"), (a + 1))).as("col")) + checkAnalysis(plan, expected) + } + + test("Analysis may leave unnecassary aliases") { + val att1 = testRelation.output.head + var plan = testRelation.select( + CreateStruct(Seq(att1, ((att1.as("aa")) + 1).as("a_plus_1"))).as("col"), + att1 + ) + val prevPlan = getAnalyzer(true).execute(plan) + plan = prevPlan.select(CreateArray(Seq( + CreateStruct(Seq(att1, (att1 + 1).as("a_plus_1"))).as("col1"), + /** alias should be eliminated by [[CleanupAliases]] */ + "col".attr.as("col2") + )).as("arr")) + plan = getAnalyzer(true).execute(plan) + + val expectedPlan = prevPlan.select( + CreateArray(Seq( + CreateNamedStruct(Seq( + Literal(att1.name), att1, + Literal("a_plus_1"), (att1 + 1))), + 'col.struct(prevPlan.output(0).dataType.asInstanceOf[StructType]).notNull + )).as("arr") + ) + + checkAnalysis(plan, expectedPlan) } test("SPARK-10534: resolve attribute references in order by clause") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala index 0c307b2b8576..c21c6de32c0b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala @@ -243,7 +243,6 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { val b = AttributeReference("b", IntegerType)() checkMetadata(CreateStruct(Seq(a, b))) checkMetadata(CreateNamedStruct(Seq("a", a, "b", b))) - checkMetadata(CreateStructUnsafe(Seq(a, b))) checkMetadata(CreateNamedStructUnsafe(Seq("a", a, "b", b))) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 249408e0fbce..7a131b30eafd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -186,6 +186,9 @@ class Column(val expr: Expression) extends Logging { case a: AggregateExpression if a.aggregateFunction.isInstanceOf[TypedAggregateExpression] => UnresolvedAlias(a, Some(Column.generateAlias)) + // Wait until the struct is resolved. This will generate a nicer looking alias. + case struct: CreateNamedStructLike => UnresolvedAlias(struct) + case expr: Expression => Alias(expr, usePrettyExpression(expr).sql)() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala index f873f34a845e..6141fab4aff0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala @@ -137,7 +137,7 @@ object ColumnStatStruct { private def numTrues(e: Expression): Expression = Sum(If(e, one, zero)) private def numFalses(e: Expression): Expression = Sum(If(Not(e), one, zero)) - private def getStruct(exprs: Seq[Expression]): CreateStruct = { + private def getStruct(exprs: Seq[Expression]): CreateNamedStruct = { CreateStruct(exprs.map { expr: Expression => expr.transformUp { case af: AggregateFunction => af.toAggregateExpression() @@ -168,7 +168,7 @@ object ColumnStatStruct { } } - def apply(attr: Attribute, relativeSD: Double): CreateStruct = attr.dataType match { + def apply(attr: Attribute, relativeSD: Double): CreateNamedStruct = attr.dataType match { // Use aggregate functions to compute statistics we need. case _: NumericType | TimestampType | DateType => getStruct(numericColumnStat(attr, relativeSD)) case StringType => getStruct(stringColumnStat(attr, relativeSD)) diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index a91f04e098b1..af6c930d64b7 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -87,7 +87,7 @@ struct -- !query 9 SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1 -- !query 9 schema -struct> +struct> -- !query 9 output diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 6eb571b91ffa..90000445dffb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -190,6 +190,12 @@ private[hive] class TestHiveSparkSession( new File(Thread.currentThread().getContextClassLoader.getResource(path).getFile) } + private def quoteHiveFile(path : String) = if (Utils.isWindows) { + getHiveFile(path).getPath.replace('\\', '/') + } else { + getHiveFile(path).getPath + } + def getWarehousePath(): String = { val tempConf = new SQLConf sc.conf.getAll.foreach { case (k, v) => tempConf.setConfString(k, v) } @@ -225,16 +231,16 @@ private[hive] class TestHiveSparkSession( val hiveQTestUtilTables: Seq[TestTable] = Seq( TestTable("src", "CREATE TABLE src (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' INTO TABLE src".cmd), + s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' INTO TABLE src".cmd), TestTable("src1", "CREATE TABLE src1 (key INT, value STRING)".cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), + s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv3.txt")}' INTO TABLE src1".cmd), TestTable("srcpart", () => { sql( "CREATE TABLE srcpart (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { sql( - s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' + s"""LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') """.stripMargin) } @@ -244,7 +250,7 @@ private[hive] class TestHiveSparkSession( "CREATE TABLE srcpart1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr INT)") for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- 11 to 12) { sql( - s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' + s"""LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/kv1.txt")}' |OVERWRITE INTO TABLE srcpart1 PARTITION (ds='$ds',hr='$hr') """.stripMargin) } @@ -269,7 +275,7 @@ private[hive] class TestHiveSparkSession( sql( s""" - |LOAD DATA LOCAL INPATH '${getHiveFile("data/files/complex.seq")}' + |LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/complex.seq")}' |INTO TABLE src_thrift """.stripMargin) }), @@ -308,7 +314,7 @@ private[hive] class TestHiveSparkSession( |) """.stripMargin.cmd, s""" - |LOAD DATA LOCAL INPATH '${getHiveFile("data/files/episodes.avro")}' + |LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/episodes.avro")}' |INTO TABLE episodes """.stripMargin.cmd ), @@ -379,7 +385,7 @@ private[hive] class TestHiveSparkSession( TestTable("src_json", s"""CREATE TABLE src_json (json STRING) STORED AS TEXTFILE """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/json.txt")}' INTO TABLE src_json".cmd) + s"LOAD DATA LOCAL INPATH '${quoteHiveFile("data/files/json.txt")}' INTO TABLE src_json".cmd) ) hiveQTestUtilTables.foreach(registerTestTable) diff --git a/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql index de0116a4dcba..cdda29af50e3 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql @@ -7,4 +7,4 @@ having b.key in (select a.key where a.value > 'val_9' and a.value = min(b.value)) order by b.key -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_5`) AS `gen_attr_1`, min(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (struct(`gen_attr_0`, `gen_attr_4`) IN (SELECT `gen_attr_6` AS `_c0`, `gen_attr_7` AS `_c1` FROM (SELECT `gen_attr_2` AS `gen_attr_6`, `gen_attr_3` AS `gen_attr_7` FROM (SELECT `gen_attr_2`, `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_3` > 'val_9')) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC NULLS FIRST) AS b +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_5`) AS `gen_attr_1`, min(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (named_struct('gen_attr_0', `gen_attr_0`, 'gen_attr_4', `gen_attr_4`) IN (SELECT `gen_attr_6` AS `_c0`, `gen_attr_7` AS `_c1` FROM (SELECT `gen_attr_2` AS `gen_attr_6`, `gen_attr_3` AS `gen_attr_7` FROM (SELECT `gen_attr_2`, `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_3` > 'val_9')) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC NULLS FIRST) AS b diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index c7f10e569fa4..12d18dc87ceb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst import java.nio.charset.StandardCharsets import java.nio.file.{Files, NoSuchFileException, Paths} +import scala.io.Source import scala.util.control.NonFatal import org.apache.spark.sql.Column @@ -109,12 +110,15 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { Files.write(path, answerText.getBytes(StandardCharsets.UTF_8)) } else { val goldenFileName = s"sqlgen/$answerFile.sql" - val resourceFile = getClass.getClassLoader.getResource(goldenFileName) - if (resourceFile == null) { + val resourceStream = getClass.getClassLoader.getResourceAsStream(goldenFileName) + if (resourceStream == null) { throw new NoSuchFileException(goldenFileName) } - val path = resourceFile.getPath - val answerText = new String(Files.readAllBytes(Paths.get(path)), StandardCharsets.UTF_8) + val answerText = try { + Source.fromInputStream(resourceStream).mkString + } finally { + resourceStream.close + } val sqls = answerText.split(separator) assert(sqls.length == 2, "Golden sql files should have a separator.") val expectedSQL = sqls(1).trim() From 4af0ce2d96de3397c9bc05684cad290a52486577 Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Wed, 2 Nov 2016 11:29:26 -0700 Subject: [PATCH 048/132] [SPARK-17683][SQL] Support ArrayType in Literal.apply ## What changes were proposed in this pull request? This pr is to add pattern-matching entries for array data in `Literal.apply`. ## How was this patch tested? Added tests in `LiteralExpressionSuite`. Author: Takeshi YAMAMURO Closes #15257 from maropu/SPARK-17683. --- .../sql/catalyst/expressions/literals.scala | 57 ++++++++++++++++++- .../expressions/LiteralExpressionSuite.scala | 27 ++++++++- 2 files changed, 82 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index a597a17aadd9..1985e68c94e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -17,14 +17,25 @@ package org.apache.spark.sql.catalyst.expressions +import java.lang.{Boolean => JavaBoolean} +import java.lang.{Byte => JavaByte} +import java.lang.{Double => JavaDouble} +import java.lang.{Float => JavaFloat} +import java.lang.{Integer => JavaInteger} +import java.lang.{Long => JavaLong} +import java.lang.{Short => JavaShort} +import java.math.{BigDecimal => JavaBigDecimal} import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import java.util import java.util.Objects import javax.xml.bind.DatatypeConverter +import scala.math.{BigDecimal, BigInt} + import org.json4s.JsonAST._ +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -46,12 +57,17 @@ object Literal { case s: String => Literal(UTF8String.fromString(s), StringType) case b: Boolean => Literal(b, BooleanType) case d: BigDecimal => Literal(Decimal(d), DecimalType(Math.max(d.precision, d.scale), d.scale)) - case d: java.math.BigDecimal => + case d: JavaBigDecimal => Literal(Decimal(d), DecimalType(Math.max(d.precision, d.scale), d.scale())) case d: Decimal => Literal(d, DecimalType(Math.max(d.precision, d.scale), d.scale)) case t: Timestamp => Literal(DateTimeUtils.fromJavaTimestamp(t), TimestampType) case d: Date => Literal(DateTimeUtils.fromJavaDate(d), DateType) case a: Array[Byte] => Literal(a, BinaryType) + case a: Array[_] => + val elementType = componentTypeToDataType(a.getClass.getComponentType()) + val dataType = ArrayType(elementType) + val convert = CatalystTypeConverters.createToCatalystConverter(dataType) + Literal(convert(a), dataType) case i: CalendarInterval => Literal(i, CalendarIntervalType) case null => Literal(null, NullType) case v: Literal => v @@ -59,6 +75,45 @@ object Literal { throw new RuntimeException("Unsupported literal type " + v.getClass + " " + v) } + /** + * Returns the Spark SQL DataType for a given class object. Since this type needs to be resolved + * in runtime, we use match-case idioms for class objects here. However, there are similar + * functions in other files (e.g., HiveInspectors), so these functions need to merged into one. + */ + private[this] def componentTypeToDataType(clz: Class[_]): DataType = clz match { + // primitive types + case JavaShort.TYPE => ShortType + case JavaInteger.TYPE => IntegerType + case JavaLong.TYPE => LongType + case JavaDouble.TYPE => DoubleType + case JavaByte.TYPE => ByteType + case JavaFloat.TYPE => FloatType + case JavaBoolean.TYPE => BooleanType + + // java classes + case _ if clz == classOf[Date] => DateType + case _ if clz == classOf[Timestamp] => TimestampType + case _ if clz == classOf[JavaBigDecimal] => DecimalType.SYSTEM_DEFAULT + case _ if clz == classOf[Array[Byte]] => BinaryType + case _ if clz == classOf[JavaShort] => ShortType + case _ if clz == classOf[JavaInteger] => IntegerType + case _ if clz == classOf[JavaLong] => LongType + case _ if clz == classOf[JavaDouble] => DoubleType + case _ if clz == classOf[JavaByte] => ByteType + case _ if clz == classOf[JavaFloat] => FloatType + case _ if clz == classOf[JavaBoolean] => BooleanType + + // other scala classes + case _ if clz == classOf[String] => StringType + case _ if clz == classOf[BigInt] => DecimalType.SYSTEM_DEFAULT + case _ if clz == classOf[BigDecimal] => DecimalType.SYSTEM_DEFAULT + case _ if clz == classOf[CalendarInterval] => CalendarIntervalType + + case _ if clz.isArray => ArrayType(componentTypeToDataType(clz.getComponentType)) + + case _ => throw new AnalysisException(s"Unsupported component type $clz in arrays") + } + /** * Constructs a [[Literal]] of [[ObjectType]], for example when you need to pass an object * into code generation. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala index 450222d8cbba..4af4da8a9f0c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala @@ -21,6 +21,7 @@ import java.nio.charset.StandardCharsets import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -43,6 +44,7 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Literal.create(null, TimestampType), null) checkEvaluation(Literal.create(null, CalendarIntervalType), null) checkEvaluation(Literal.create(null, ArrayType(ByteType, true)), null) + checkEvaluation(Literal.create(null, ArrayType(StringType, true)), null) checkEvaluation(Literal.create(null, MapType(StringType, IntegerType)), null) checkEvaluation(Literal.create(null, StructType(Seq.empty)), null) } @@ -122,5 +124,28 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { } } - // TODO(davies): add tests for ArrayType, MapType and StructType + test("array") { + def checkArrayLiteral(a: Array[_], elementType: DataType): Unit = { + val toCatalyst = (a: Array[_], elementType: DataType) => { + CatalystTypeConverters.createToCatalystConverter(ArrayType(elementType))(a) + } + checkEvaluation(Literal(a), toCatalyst(a, elementType)) + } + checkArrayLiteral(Array(1, 2, 3), IntegerType) + checkArrayLiteral(Array("a", "b", "c"), StringType) + checkArrayLiteral(Array(1.0, 4.0), DoubleType) + checkArrayLiteral(Array(CalendarInterval.MICROS_PER_DAY, CalendarInterval.MICROS_PER_HOUR), + CalendarIntervalType) + } + + test("unsupported types (map and struct) in literals") { + def checkUnsupportedTypeInLiteral(v: Any): Unit = { + val errMsgMap = intercept[RuntimeException] { + Literal(v) + } + assert(errMsgMap.getMessage.startsWith("Unsupported literal type")) + } + checkUnsupportedTypeInLiteral(Map("key1" -> 1, "key2" -> 2)) + checkUnsupportedTypeInLiteral(("mike", 29, 1.0)) + } } From 742e0fea5391857964e90d396641ecf95cac4248 Mon Sep 17 00:00:00 2001 From: buzhihuojie Date: Wed, 2 Nov 2016 11:36:20 -0700 Subject: [PATCH 049/132] [SPARK-17895] Improve doc for rangeBetween and rowsBetween ## What changes were proposed in this pull request? Copied description for row and range based frame boundary from https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala#L56 Added examples to show different behavior of rangeBetween and rowsBetween when involving duplicate values. Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request. Author: buzhihuojie Closes #15727 from david-weiluo-ren/improveDocForRangeAndRowsBetween. --- .../apache/spark/sql/expressions/Window.scala | 55 +++++++++++++++++++ .../spark/sql/expressions/WindowSpec.scala | 55 +++++++++++++++++++ 2 files changed, 110 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala index 0b26d863cac5..327bc379d413 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -121,6 +121,32 @@ object Window { * and [[Window.currentRow]] to specify special boundary values, rather than using integral * values directly. * + * A row based boundary is based on the position of the row within the partition. + * An offset indicates the number of rows above or below the current row, the frame for the + * current row starts or ends. For instance, given a row based sliding frame with a lower bound + * offset of -1 and a upper bound offset of +2. The frame for row with index 5 would range from + * index 4 to index 6. + * + * {{{ + * import org.apache.spark.sql.expressions.Window + * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) + * .toDF("id", "category") + * df.withColumn("sum", + * sum('id) over Window.partitionBy('category).orderBy('id).rowsBetween(0,1)) + * .show() + * + * +---+--------+---+ + * | id|category|sum| + * +---+--------+---+ + * | 1| b| 3| + * | 2| b| 5| + * | 3| b| 3| + * | 1| a| 2| + * | 1| a| 3| + * | 2| a| 2| + * +---+--------+---+ + * }}} + * * @param start boundary start, inclusive. The frame is unbounded if this is * the minimum long value ([[Window.unboundedPreceding]]). * @param end boundary end, inclusive. The frame is unbounded if this is the @@ -144,6 +170,35 @@ object Window { * and [[Window.currentRow]] to specify special boundary values, rather than using integral * values directly. * + * A range based boundary is based on the actual value of the ORDER BY + * expression(s). An offset is used to alter the value of the ORDER BY expression, for + * instance if the current order by expression has a value of 10 and the lower bound offset + * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a + * number of constraints on the ORDER BY expressions: there can be only one expression and this + * expression must have a numerical data type. An exception can be made when the offset is 0, + * because no value modification is needed, in this case multiple and non-numeric ORDER BY + * expression are allowed. + * + * {{{ + * import org.apache.spark.sql.expressions.Window + * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) + * .toDF("id", "category") + * df.withColumn("sum", + * sum('id) over Window.partitionBy('category).orderBy('id).rangeBetween(0,1)) + * .show() + * + * +---+--------+---+ + * | id|category|sum| + * +---+--------+---+ + * | 1| b| 3| + * | 2| b| 5| + * | 3| b| 3| + * | 1| a| 4| + * | 1| a| 4| + * | 2| a| 2| + * +---+--------+---+ + * }}} + * * @param start boundary start, inclusive. The frame is unbounded if this is * the minimum long value ([[Window.unboundedPreceding]]). * @param end boundary end, inclusive. The frame is unbounded if this is the diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala index 1e85b6e7881a..4a8ce695bd4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala @@ -89,6 +89,32 @@ class WindowSpec private[sql]( * and [[Window.currentRow]] to specify special boundary values, rather than using integral * values directly. * + * A row based boundary is based on the position of the row within the partition. + * An offset indicates the number of rows above or below the current row, the frame for the + * current row starts or ends. For instance, given a row based sliding frame with a lower bound + * offset of -1 and a upper bound offset of +2. The frame for row with index 5 would range from + * index 4 to index 6. + * + * {{{ + * import org.apache.spark.sql.expressions.Window + * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) + * .toDF("id", "category") + * df.withColumn("sum", + * sum('id) over Window.partitionBy('category).orderBy('id).rowsBetween(0,1)) + * .show() + * + * +---+--------+---+ + * | id|category|sum| + * +---+--------+---+ + * | 1| b| 3| + * | 2| b| 5| + * | 3| b| 3| + * | 1| a| 2| + * | 1| a| 3| + * | 2| a| 2| + * +---+--------+---+ + * }}} + * * @param start boundary start, inclusive. The frame is unbounded if this is * the minimum long value ([[Window.unboundedPreceding]]). * @param end boundary end, inclusive. The frame is unbounded if this is the @@ -111,6 +137,35 @@ class WindowSpec private[sql]( * and [[Window.currentRow]] to specify special boundary values, rather than using integral * values directly. * + * A range based boundary is based on the actual value of the ORDER BY + * expression(s). An offset is used to alter the value of the ORDER BY expression, for + * instance if the current order by expression has a value of 10 and the lower bound offset + * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a + * number of constraints on the ORDER BY expressions: there can be only one expression and this + * expression must have a numerical data type. An exception can be made when the offset is 0, + * because no value modification is needed, in this case multiple and non-numeric ORDER BY + * expression are allowed. + * + * {{{ + * import org.apache.spark.sql.expressions.Window + * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) + * .toDF("id", "category") + * df.withColumn("sum", + * sum('id) over Window.partitionBy('category).orderBy('id).rangeBetween(0,1)) + * .show() + * + * +---+--------+---+ + * | id|category|sum| + * +---+--------+---+ + * | 1| b| 3| + * | 2| b| 5| + * | 3| b| 3| + * | 1| a| 4| + * | 1| a| 4| + * | 2| a| 2| + * +---+--------+---+ + * }}} + * * @param start boundary start, inclusive. The frame is unbounded if this is * the minimum long value ([[Window.unboundedPreceding]]). * @param end boundary end, inclusive. The frame is unbounded if this is the From 02f203107b8eda1f1576e36c4f12b0e3bc5e910e Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 2 Nov 2016 11:41:49 -0700 Subject: [PATCH 050/132] [SPARK-14393][SQL] values generated by non-deterministic functions shouldn't change after coalesce or union ## What changes were proposed in this pull request? When a user appended a column using a "nondeterministic" function to a DataFrame, e.g., `rand`, `randn`, and `monotonically_increasing_id`, the expected semantic is the following: - The value in each row should remain unchanged, as if we materialize the column immediately, regardless of later DataFrame operations. However, since we use `TaskContext.getPartitionId` to get the partition index from the current thread, the values from nondeterministic columns might change if we call `union` or `coalesce` after. `TaskContext.getPartitionId` returns the partition index of the current Spark task, which might not be the corresponding partition index of the DataFrame where we defined the column. See the unit tests below or JIRA for examples. This PR uses the partition index from `RDD.mapPartitionWithIndex` instead of `TaskContext` and fixes the partition initialization logic in whole-stage codegen, normal codegen, and codegen fallback. `initializeStatesForPartition(partitionIndex: Int)` was added to `Projection`, `Nondeterministic`, and `Predicate` (codegen) and initialized right after object creation in `mapPartitionWithIndex`. `newPredicate` now returns a `Predicate` instance rather than a function for proper initialization. ## How was this patch tested? Unit tests. (Actually I'm not very confident that this PR fixed all issues without introducing new ones ...) cc: rxin davies Author: Xiangrui Meng Closes #15567 from mengxr/SPARK-14393. --- .../main/scala/org/apache/spark/rdd/RDD.scala | 16 +++++- .../sql/catalyst/expressions/Expression.scala | 19 +++++-- .../catalyst/expressions/InputFileName.scala | 2 +- .../MonotonicallyIncreasingID.scala | 11 ++-- .../sql/catalyst/expressions/Projection.scala | 22 +++++--- .../expressions/SparkPartitionID.scala | 13 +++-- .../expressions/codegen/CodeGenerator.scala | 14 +++++ .../expressions/codegen/CodegenFallback.scala | 18 +++++-- .../codegen/GenerateMutableProjection.scala | 4 ++ .../codegen/GeneratePredicate.scala | 18 +++++-- .../codegen/GenerateSafeProjection.scala | 4 ++ .../codegen/GenerateUnsafeProjection.scala | 4 ++ .../sql/catalyst/expressions/package.scala | 10 +++- .../sql/catalyst/expressions/predicates.scala | 4 -- .../expressions/randomExpressions.scala | 14 ++--- .../sql/catalyst/optimizer/Optimizer.scala | 1 + .../expressions/ExpressionEvalHelper.scala | 5 +- .../CodegenExpressionCachingSuite.scala | 13 +++-- .../sql/execution/DataSourceScanExec.scala | 6 ++- .../spark/sql/execution/ExistingRDD.scala | 3 +- .../spark/sql/execution/GenerateExec.scala | 3 +- .../spark/sql/execution/SparkPlan.scala | 4 +- .../sql/execution/WholeStageCodegenExec.scala | 8 ++- .../execution/basicPhysicalOperators.scala | 8 +-- .../columnar/InMemoryTableScanExec.scala | 5 +- .../joins/BroadcastNestedLoopJoinExec.scala | 7 +-- .../joins/CartesianProductExec.scala | 8 +-- .../spark/sql/execution/joins/HashJoin.scala | 2 +- .../execution/joins/SortMergeJoinExec.scala | 2 +- .../apache/spark/sql/execution/objects.scala | 6 ++- .../spark/sql/DataFrameFunctionsSuite.scala | 52 +++++++++++++++++++ .../hive/execution/HiveTableScanExec.scala | 3 +- 32 files changed, 231 insertions(+), 78 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index db535de9e9bb..e018af35cb18 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -788,14 +788,26 @@ abstract class RDD[T: ClassTag]( } /** - * [performance] Spark's internal mapPartitions method which skips closure cleaning. It is a - * performance API to be used carefully only if we are sure that the RDD elements are + * [performance] Spark's internal mapPartitionsWithIndex method that skips closure cleaning. + * It is a performance API to be used carefully only if we are sure that the RDD elements are * serializable and don't require closure cleaning. * * @param preservesPartitioning indicates whether the input function preserves the partitioner, * which should be `false` unless this is a pair RDD and the input function doesn't modify * the keys. */ + private[spark] def mapPartitionsWithIndexInternal[U: ClassTag]( + f: (Int, Iterator[T]) => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = withScope { + new MapPartitionsRDD( + this, + (context: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter), + preservesPartitioning) + } + + /** + * [performance] Spark's internal mapPartitions method that skips closure cleaning. + */ private[spark] def mapPartitionsInternal[U: ClassTag]( f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = withScope { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 9edc1ceff26a..726a231fd814 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -272,17 +272,28 @@ trait Nondeterministic extends Expression { final override def deterministic: Boolean = false final override def foldable: Boolean = false + @transient private[this] var initialized = false - final def setInitialValues(): Unit = { - initInternal() + /** + * Initializes internal states given the current partition index and mark this as initialized. + * Subclasses should override [[initializeInternal()]]. + */ + final def initialize(partitionIndex: Int): Unit = { + initializeInternal(partitionIndex) initialized = true } - protected def initInternal(): Unit + protected def initializeInternal(partitionIndex: Int): Unit + /** + * @inheritdoc + * Throws an exception if [[initialize()]] is not called yet. + * Subclasses should override [[evalInternal()]]. + */ final override def eval(input: InternalRow = null): Any = { - require(initialized, "nondeterministic expression should be initialized before evaluate") + require(initialized, + s"Nondeterministic expression ${this.getClass.getName} should be initialized before eval.") evalInternal(input) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala index 96929ecf5637..b6c12c535111 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InputFileName.scala @@ -37,7 +37,7 @@ case class InputFileName() extends LeafExpression with Nondeterministic { override def prettyName: String = "input_file_name" - override protected def initInternal(): Unit = {} + override protected def initializeInternal(partitionIndex: Int): Unit = {} override protected def evalInternal(input: InternalRow): UTF8String = { InputFileNameHolder.getInputFileName() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala index 5b4922e0cf2b..72b8dcca26e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/MonotonicallyIncreasingID.scala @@ -50,9 +50,9 @@ case class MonotonicallyIncreasingID() extends LeafExpression with Nondeterminis @transient private[this] var partitionMask: Long = _ - override protected def initInternal(): Unit = { + override protected def initializeInternal(partitionIndex: Int): Unit = { count = 0L - partitionMask = TaskContext.getPartitionId().toLong << 33 + partitionMask = partitionIndex.toLong << 33 } override def nullable: Boolean = false @@ -68,9 +68,10 @@ case class MonotonicallyIncreasingID() extends LeafExpression with Nondeterminis override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val countTerm = ctx.freshName("count") val partitionMaskTerm = ctx.freshName("partitionMask") - ctx.addMutableState(ctx.JAVA_LONG, countTerm, s"$countTerm = 0L;") - ctx.addMutableState(ctx.JAVA_LONG, partitionMaskTerm, - s"$partitionMaskTerm = ((long) org.apache.spark.TaskContext.getPartitionId()) << 33;") + ctx.addMutableState(ctx.JAVA_LONG, countTerm, "") + ctx.addMutableState(ctx.JAVA_LONG, partitionMaskTerm, "") + ctx.addPartitionInitializationStatement(s"$countTerm = 0L;") + ctx.addPartitionInitializationStatement(s"$partitionMaskTerm = ((long) partitionIndex) << 33;") ev.copy(code = s""" final ${ctx.javaType(dataType)} ${ev.value} = $partitionMaskTerm + $countTerm; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 03e054d09851..476e37e6a9ba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.types.{DataType, StructType} /** * A [[Projection]] that is calculated by calling the `eval` of each of the specified expressions. + * * @param expressions a sequence of expressions that determine the value of each column of the * output row. */ @@ -30,10 +31,12 @@ class InterpretedProjection(expressions: Seq[Expression]) extends Projection { def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = this(expressions.map(BindReferences.bindReference(_, inputSchema))) - expressions.foreach(_.foreach { - case n: Nondeterministic => n.setInitialValues() - case _ => - }) + override def initialize(partitionIndex: Int): Unit = { + expressions.foreach(_.foreach { + case n: Nondeterministic => n.initialize(partitionIndex) + case _ => + }) + } // null check is required for when Kryo invokes the no-arg constructor. protected val exprArray = if (expressions != null) expressions.toArray else null @@ -54,6 +57,7 @@ class InterpretedProjection(expressions: Seq[Expression]) extends Projection { /** * A [[MutableProjection]] that is calculated by calling `eval` on each of the specified * expressions. + * * @param expressions a sequence of expressions that determine the value of each column of the * output row. */ @@ -63,10 +67,12 @@ case class InterpretedMutableProjection(expressions: Seq[Expression]) extends Mu private[this] val buffer = new Array[Any](expressions.size) - expressions.foreach(_.foreach { - case n: Nondeterministic => n.setInitialValues() - case _ => - }) + override def initialize(partitionIndex: Int): Unit = { + expressions.foreach(_.foreach { + case n: Nondeterministic => n.initialize(partitionIndex) + case _ => + }) + } private[this] val exprArray = expressions.toArray private[this] var mutableRow: InternalRow = new GenericInternalRow(exprArray.length) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala index 1f675d5b0727..6bef473cac06 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SparkPartitionID.scala @@ -17,16 +17,15 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.types.{DataType, IntegerType} /** - * Expression that returns the current partition id of the Spark task. + * Expression that returns the current partition id. */ @ExpressionDescription( - usage = "_FUNC_() - Returns the current partition id of the Spark task", + usage = "_FUNC_() - Returns the current partition id", extended = "> SELECT _FUNC_();\n 0") case class SparkPartitionID() extends LeafExpression with Nondeterministic { @@ -38,16 +37,16 @@ case class SparkPartitionID() extends LeafExpression with Nondeterministic { override val prettyName = "SPARK_PARTITION_ID" - override protected def initInternal(): Unit = { - partitionId = TaskContext.getPartitionId() + override protected def initializeInternal(partitionIndex: Int): Unit = { + partitionId = partitionIndex } override protected def evalInternal(input: InternalRow): Int = partitionId override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val idTerm = ctx.freshName("partitionId") - ctx.addMutableState(ctx.JAVA_INT, idTerm, - s"$idTerm = org.apache.spark.TaskContext.getPartitionId();") + ctx.addMutableState(ctx.JAVA_INT, idTerm, "") + ctx.addPartitionInitializationStatement(s"$idTerm = partitionIndex;") ev.copy(code = s"final ${ctx.javaType(dataType)} ${ev.value} = $idTerm;", isNull = "false") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 6cab50ae1bf8..9c3c6d3b2a7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -184,6 +184,20 @@ class CodegenContext { splitExpressions(initCodes, "init", Nil) } + /** + * Code statements to initialize states that depend on the partition index. + * An integer `partitionIndex` will be made available within the scope. + */ + val partitionInitializationStatements: mutable.ArrayBuffer[String] = mutable.ArrayBuffer.empty + + def addPartitionInitializationStatement(statement: String): Unit = { + partitionInitializationStatements += statement + } + + def initPartition(): String = { + partitionInitializationStatements.mkString("\n") + } + /** * Holding all the functions those will be added into generated class. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala index 6a5a3e7933ee..0322d1dd6a9f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenFallback.scala @@ -25,15 +25,23 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, LeafExpression, No trait CodegenFallback extends Expression { protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - foreach { - case n: Nondeterministic => n.setInitialValues() - case _ => - } - // LeafNode does not need `input` val input = if (this.isInstanceOf[LeafExpression]) "null" else ctx.INPUT_ROW val idx = ctx.references.length ctx.references += this + var childIndex = idx + this.foreach { + case n: Nondeterministic => + // This might add the current expression twice, but it won't hurt. + ctx.references += n + childIndex += 1 + ctx.addPartitionInitializationStatement( + s""" + |((Nondeterministic) references[$childIndex]) + | .initialize(partitionIndex); + """.stripMargin) + case _ => + } val objectTerm = ctx.freshName("obj") val placeHolder = ctx.registerComment(this.toString) if (nullable) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index 5c4b56b0b224..4d732445544a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -111,6 +111,10 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP ${ctx.initMutableStates()} } + public void initialize(int partitionIndex) { + ${ctx.initPartition()} + } + ${ctx.declareAddedFunctions()} public ${classOf[BaseMutableProjection].getName} target(InternalRow row) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala index 39aa7b17de6c..dcd1ed96a298 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -25,19 +25,26 @@ import org.apache.spark.sql.catalyst.expressions._ */ abstract class Predicate { def eval(r: InternalRow): Boolean + + /** + * Initializes internal states given the current partition index. + * This is used by nondeterministic expressions to set initial states. + * The default implementation does nothing. + */ + def initialize(partitionIndex: Int): Unit = {} } /** * Generates bytecode that evaluates a boolean [[Expression]] on a given input [[InternalRow]]. */ -object GeneratePredicate extends CodeGenerator[Expression, (InternalRow) => Boolean] { +object GeneratePredicate extends CodeGenerator[Expression, Predicate] { protected def canonicalize(in: Expression): Expression = ExpressionCanonicalizer.execute(in) protected def bind(in: Expression, inputSchema: Seq[Attribute]): Expression = BindReferences.bindReference(in, inputSchema) - protected def create(predicate: Expression): ((InternalRow) => Boolean) = { + protected def create(predicate: Expression): Predicate = { val ctx = newCodeGenContext() val eval = predicate.genCode(ctx) @@ -55,6 +62,10 @@ object GeneratePredicate extends CodeGenerator[Expression, (InternalRow) => Bool ${ctx.initMutableStates()} } + public void initialize(int partitionIndex) { + ${ctx.initPartition()} + } + ${ctx.declareAddedFunctions()} public boolean eval(InternalRow ${ctx.INPUT_ROW}) { @@ -67,7 +78,6 @@ object GeneratePredicate extends CodeGenerator[Expression, (InternalRow) => Bool new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) logDebug(s"Generated predicate '$predicate':\n${CodeFormatter.format(code)}") - val p = CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[Predicate] - (r: InternalRow) => p.eval(r) + CodeGenerator.compile(code).generate(ctx.references.toArray).asInstanceOf[Predicate] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index 2773e1a66621..b1cb6edefb85 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -173,6 +173,10 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] ${ctx.initMutableStates()} } + public void initialize(int partitionIndex) { + ${ctx.initPartition()} + } + ${ctx.declareAddedFunctions()} public java.lang.Object apply(java.lang.Object _i) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 7cc45372daa5..7e4c9089a2cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -380,6 +380,10 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro ${ctx.initMutableStates()} } + public void initialize(int partitionIndex) { + ${ctx.initPartition()} + } + ${ctx.declareAddedFunctions()} // Scala.Function1 need this diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index 1510a4796683..1b00c9e79da2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -64,7 +64,15 @@ package object expressions { * column of the new row. If the schema of the input row is specified, then the given expression * will be bound to that schema. */ - abstract class Projection extends (InternalRow => InternalRow) + abstract class Projection extends (InternalRow => InternalRow) { + + /** + * Initializes internal states given the current partition index. + * This is used by nondeterministic expressions to set initial states. + * The default implementation does nothing. + */ + def initialize(partitionIndex: Int): Unit = {} + } /** * Converts a [[InternalRow]] to another Row given a sequence of expression that define each diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 9394e39aadd9..c941a576d00d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -31,10 +31,6 @@ object InterpretedPredicate { create(BindReferences.bindReference(expression, inputSchema)) def create(expression: Expression): (InternalRow => Boolean) = { - expression.foreach { - case n: Nondeterministic => n.setInitialValues() - case _ => - } (r: InternalRow) => expression.eval(r).asInstanceOf[Boolean] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala index ca200768b228..e09029f5aab9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala @@ -42,8 +42,8 @@ abstract class RDG extends LeafExpression with Nondeterministic { */ @transient protected var rng: XORShiftRandom = _ - override protected def initInternal(): Unit = { - rng = new XORShiftRandom(seed + TaskContext.getPartitionId) + override protected def initializeInternal(partitionIndex: Int): Unit = { + rng = new XORShiftRandom(seed + partitionIndex) } override def nullable: Boolean = false @@ -70,8 +70,9 @@ case class Rand(seed: Long) extends RDG { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val rngTerm = ctx.freshName("rng") val className = classOf[XORShiftRandom].getName - ctx.addMutableState(className, rngTerm, - s"$rngTerm = new $className(${seed}L + org.apache.spark.TaskContext.getPartitionId());") + ctx.addMutableState(className, rngTerm, "") + ctx.addPartitionInitializationStatement( + s"$rngTerm = new $className(${seed}L + partitionIndex);") ev.copy(code = s""" final ${ctx.javaType(dataType)} ${ev.value} = $rngTerm.nextDouble();""", isNull = "false") } @@ -93,8 +94,9 @@ case class Randn(seed: Long) extends RDG { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val rngTerm = ctx.freshName("rng") val className = classOf[XORShiftRandom].getName - ctx.addMutableState(className, rngTerm, - s"$rngTerm = new $className(${seed}L + org.apache.spark.TaskContext.getPartitionId());") + ctx.addMutableState(className, rngTerm, "") + ctx.addPartitionInitializationStatement( + s"$rngTerm = new $className(${seed}L + partitionIndex);") ev.copy(code = s""" final ${ctx.javaType(dataType)} ${ev.value} = $rngTerm.nextGaussian();""", isNull = "false") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index e5e2cd7d27d1..b6ad5db74e3c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1060,6 +1060,7 @@ object ConvertToLocalRelation extends Rule[LogicalPlan] { case Project(projectList, LocalRelation(output, data)) if !projectList.exists(hasUnevaluableExpr) => val projection = new InterpretedProjection(projectList, output) + projection.initialize(0) LocalRelation(projectList.map(_.toAttribute), data.map(projection)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index f0c149c02b9a..9ceb70918541 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -75,7 +75,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { protected def evaluate(expression: Expression, inputRow: InternalRow = EmptyRow): Any = { expression.foreach { - case n: Nondeterministic => n.setInitialValues() + case n: Nondeterministic => n.initialize(0) case _ => } expression.eval(inputRow) @@ -121,6 +121,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { val plan = generateProject( GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil), expression) + plan.initialize(0) val actual = plan(inputRow).get(0, expression.dataType) if (!checkResult(actual, expected)) { @@ -182,12 +183,14 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { var plan = generateProject( GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil), expression) + plan.initialize(0) var actual = plan(inputRow).get(0, expression.dataType) assert(checkResult(actual, expected)) plan = generateProject( GenerateUnsafeProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil), expression) + plan.initialize(0) actual = FromUnsafeProjection(expression.dataType :: Nil)( plan(inputRow)).get(0, expression.dataType) assert(checkResult(actual, expected)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenExpressionCachingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenExpressionCachingSuite.scala index 06dc3bd33b90..fe5cb8eda824 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenExpressionCachingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenExpressionCachingSuite.scala @@ -31,19 +31,22 @@ class CodegenExpressionCachingSuite extends SparkFunSuite { // Use an Add to wrap two of them together in case we only initialize the top level expressions. val expr = And(NondeterministicExpression(), NondeterministicExpression()) val instance = UnsafeProjection.create(Seq(expr)) + instance.initialize(0) assert(instance.apply(null).getBoolean(0) === false) } test("GenerateMutableProjection should initialize expressions") { val expr = And(NondeterministicExpression(), NondeterministicExpression()) val instance = GenerateMutableProjection.generate(Seq(expr)) + instance.initialize(0) assert(instance.apply(null).getBoolean(0) === false) } test("GeneratePredicate should initialize expressions") { val expr = And(NondeterministicExpression(), NondeterministicExpression()) val instance = GeneratePredicate.generate(expr) - assert(instance.apply(null) === false) + instance.initialize(0) + assert(instance.eval(null) === false) } test("GenerateUnsafeProjection should not share expression instances") { @@ -73,13 +76,13 @@ class CodegenExpressionCachingSuite extends SparkFunSuite { test("GeneratePredicate should not share expression instances") { val expr1 = MutableExpression() val instance1 = GeneratePredicate.generate(expr1) - assert(instance1.apply(null) === false) + assert(instance1.eval(null) === false) val expr2 = MutableExpression() expr2.mutableState = true val instance2 = GeneratePredicate.generate(expr2) - assert(instance1.apply(null) === false) - assert(instance2.apply(null) === true) + assert(instance1.eval(null) === false) + assert(instance2.eval(null) === true) } } @@ -89,7 +92,7 @@ class CodegenExpressionCachingSuite extends SparkFunSuite { */ case class NondeterministicExpression() extends LeafExpression with Nondeterministic with CodegenFallback { - override protected def initInternal(): Unit = { } + override protected def initializeInternal(partitionIndex: Int): Unit = {} override protected def evalInternal(input: InternalRow): Any = false override def nullable: Boolean = false override def dataType: DataType = BooleanType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index fdd1fa364825..e485b52b43f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -71,8 +71,9 @@ case class RowDataSourceScanExec( val unsafeRow = if (outputUnsafeRows) { rdd } else { - rdd.mapPartitionsInternal { iter => + rdd.mapPartitionsWithIndexInternal { (index, iter) => val proj = UnsafeProjection.create(schema) + proj.initialize(index) iter.map(proj) } } @@ -284,8 +285,9 @@ case class FileSourceScanExec( val unsafeRows = { val scan = inputRDD if (needsUnsafeRowConversion) { - scan.mapPartitionsInternal { iter => + scan.mapPartitionsWithIndexInternal { (index, iter) => val proj = UnsafeProjection.create(schema) + proj.initialize(index) iter.map(proj) } } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 455fb5bfbb6f..aab087cd9871 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -190,8 +190,9 @@ case class RDDScanExec( protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") - rdd.mapPartitionsInternal { iter => + rdd.mapPartitionsWithIndexInternal { (index, iter) => val proj = UnsafeProjection.create(schema) + proj.initialize(index) iter.map { r => numOutputRows += 1 proj(r) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala index 266312956266..19fbf0c16204 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala @@ -94,8 +94,9 @@ case class GenerateExec( } val numOutputRows = longMetric("numOutputRows") - rows.mapPartitionsInternal { iter => + rows.mapPartitionsWithIndexInternal { (index, iter) => val proj = UnsafeProjection.create(output, output) + proj.initialize(index) iter.map { r => numOutputRows += 1 proj(r) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 24d0cffef82a..cadab37a449a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -29,7 +29,7 @@ import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.expressions.codegen.{Predicate => GenPredicate, _} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.SQLMetric @@ -354,7 +354,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ } protected def newPredicate( - expression: Expression, inputSchema: Seq[Attribute]): (InternalRow) => Boolean = { + expression: Expression, inputSchema: Seq[Attribute]): GenPredicate = { GeneratePredicate.generate(expression, inputSchema) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 6303483f22fd..516b9d5444d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -331,6 +331,7 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co partitionIndex = index; this.inputs = inputs; ${ctx.initMutableStates()} + ${ctx.initPartition()} } ${ctx.declareAddedFunctions()} @@ -383,10 +384,13 @@ case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with Co } else { // Right now, we support up to two input RDDs. rdds.head.zipPartitions(rdds(1)) { (leftIter, rightIter) => - val partitionIndex = TaskContext.getPartitionId() + Iterator((leftIter, rightIter)) + // a small hack to obtain the correct partition index + }.mapPartitionsWithIndex { (index, zippedIter) => + val (leftIter, rightIter) = zippedIter.next() val clazz = CodeGenerator.compile(cleanedSource) val buffer = clazz.generate(references).asInstanceOf[BufferedRowIterator] - buffer.init(partitionIndex, Array(leftIter, rightIter)) + buffer.init(index, Array(leftIter, rightIter)) new Iterator[InternalRow] { override def hasNext: Boolean = { val v = buffer.hasNext diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index a5291e0c12f8..32133f52630c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -70,9 +70,10 @@ case class ProjectExec(projectList: Seq[NamedExpression], child: SparkPlan) } protected override def doExecute(): RDD[InternalRow] = { - child.execute().mapPartitionsInternal { iter => + child.execute().mapPartitionsWithIndexInternal { (index, iter) => val project = UnsafeProjection.create(projectList, child.output, subexpressionEliminationEnabled) + project.initialize(index) iter.map(project) } } @@ -205,10 +206,11 @@ case class FilterExec(condition: Expression, child: SparkPlan) protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") - child.execute().mapPartitionsInternal { iter => + child.execute().mapPartitionsWithIndexInternal { (index, iter) => val predicate = newPredicate(condition, child.output) + predicate.initialize(0) iter.filter { row => - val r = predicate(row) + val r = predicate.eval(row) if (r) numOutputRows += 1 r } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index b87016d5a569..9028caa446e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -132,10 +132,11 @@ case class InMemoryTableScanExec( val relOutput: AttributeSeq = relation.output val buffers = relation.cachedColumnBuffers - buffers.mapPartitionsInternal { cachedBatchIterator => + buffers.mapPartitionsWithIndexInternal { (index, cachedBatchIterator) => val partitionFilter = newPredicate( partitionFilters.reduceOption(And).getOrElse(Literal(true)), schema) + partitionFilter.initialize(index) // Find the ordinals and data types of the requested columns. val (requestedColumnIndices, requestedColumnDataTypes) = @@ -147,7 +148,7 @@ case class InMemoryTableScanExec( val cachedBatchesToScan = if (inMemoryPartitionPruningEnabled) { cachedBatchIterator.filter { cachedBatch => - if (!partitionFilter(cachedBatch.stats)) { + if (!partitionFilter.eval(cachedBatch.stats)) { def statsString: String = schemaIndex.map { case (a, i) => val value = cachedBatch.stats.get(i, a.dataType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala index bfe7e3dea45d..f526a1987667 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoinExec.scala @@ -52,7 +52,7 @@ case class BroadcastNestedLoopJoinExec( UnspecifiedDistribution :: BroadcastDistribution(IdentityBroadcastMode) :: Nil } - private[this] def genResultProjection: InternalRow => InternalRow = joinType match { + private[this] def genResultProjection: UnsafeProjection = joinType match { case LeftExistence(j) => UnsafeProjection.create(output, output) case other => @@ -84,7 +84,7 @@ case class BroadcastNestedLoopJoinExec( @transient private lazy val boundCondition = { if (condition.isDefined) { - newPredicate(condition.get, streamed.output ++ broadcast.output) + newPredicate(condition.get, streamed.output ++ broadcast.output).eval _ } else { (r: InternalRow) => true } @@ -366,8 +366,9 @@ case class BroadcastNestedLoopJoinExec( } val numOutputRows = longMetric("numOutputRows") - resultRdd.mapPartitionsInternal { iter => + resultRdd.mapPartitionsWithIndexInternal { (index, iter) => val resultProj = genResultProjection + resultProj.initialize(index) iter.map { r => numOutputRows += 1 resultProj(r) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala index 15dc9b40662e..8341fe2ffd07 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProductExec.scala @@ -98,15 +98,15 @@ case class CartesianProductExec( val rightResults = right.execute().asInstanceOf[RDD[UnsafeRow]] val pair = new UnsafeCartesianRDD(leftResults, rightResults, right.output.size) - pair.mapPartitionsInternal { iter => + pair.mapPartitionsWithIndexInternal { (index, iter) => val joiner = GenerateUnsafeRowJoiner.create(left.schema, right.schema) val filtered = if (condition.isDefined) { - val boundCondition: (InternalRow) => Boolean = - newPredicate(condition.get, left.output ++ right.output) + val boundCondition = newPredicate(condition.get, left.output ++ right.output) + boundCondition.initialize(index) val joined = new JoinedRow iter.filter { r => - boundCondition(joined(r._1, r._2)) + boundCondition.eval(joined(r._1, r._2)) } } else { iter diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 05c5e2f4cd77..1aef5f686426 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -81,7 +81,7 @@ trait HashJoin { UnsafeProjection.create(streamedKeys) @transient private[this] lazy val boundCondition = if (condition.isDefined) { - newPredicate(condition.get, streamedPlan.output ++ buildPlan.output) + newPredicate(condition.get, streamedPlan.output ++ buildPlan.output).eval _ } else { (r: InternalRow) => true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index ecf7cf289f03..ca9c0ed8cec3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -101,7 +101,7 @@ case class SortMergeJoinExec( left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) => val boundCondition: (InternalRow) => Boolean = { condition.map { cond => - newPredicate(cond, left.output ++ right.output) + newPredicate(cond, left.output ++ right.output).eval _ }.getOrElse { (r: InternalRow) => true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 9df56bbf1ef8..fde3b2a52899 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -87,8 +87,9 @@ case class DeserializeToObjectExec( } override protected def doExecute(): RDD[InternalRow] = { - child.execute().mapPartitionsInternal { iter => + child.execute().mapPartitionsWithIndexInternal { (index, iter) => val projection = GenerateSafeProjection.generate(deserializer :: Nil, child.output) + projection.initialize(index) iter.map(projection) } } @@ -124,8 +125,9 @@ case class SerializeFromObjectExec( } override protected def doExecute(): RDD[InternalRow] = { - child.execute().mapPartitionsInternal { iter => + child.execute().mapPartitionsWithIndexInternal { (index, iter) => val projection = UnsafeProjection.create(serializer) + projection.initialize(index) iter.map(projection) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index 586a0fffeb7a..0e9a2c6cf7de 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -19,7 +19,13 @@ package org.apache.spark.sql import java.nio.charset.StandardCharsets +import scala.util.Random + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -406,4 +412,50 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { Seq(Row(true), Row(true)) ) } + + private def assertValuesDoNotChangeAfterCoalesceOrUnion(v: Column): Unit = { + import DataFrameFunctionsSuite.CodegenFallbackExpr + for ((codegenFallback, wholeStage) <- Seq((true, false), (false, false), (false, true))) { + val c = if (codegenFallback) { + Column(CodegenFallbackExpr(v.expr)) + } else { + v + } + withSQLConf( + (SQLConf.WHOLESTAGE_FALLBACK.key, codegenFallback.toString), + (SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, wholeStage.toString)) { + val df = spark.range(0, 4, 1, 4).withColumn("c", c) + val rows = df.collect() + val rowsAfterCoalesce = df.coalesce(2).collect() + assert(rows === rowsAfterCoalesce, "Values changed after coalesce when " + + s"codegenFallback=$codegenFallback and wholeStage=$wholeStage.") + + val df1 = spark.range(0, 2, 1, 2).withColumn("c", c) + val rows1 = df1.collect() + val df2 = spark.range(2, 4, 1, 2).withColumn("c", c) + val rows2 = df2.collect() + val rowsAfterUnion = df1.union(df2).collect() + assert(rowsAfterUnion === rows1 ++ rows2, "Values changed after union when " + + s"codegenFallback=$codegenFallback and wholeStage=$wholeStage.") + } + } + } + + test("SPARK-14393: values generated by non-deterministic functions shouldn't change after " + + "coalesce or union") { + Seq( + monotonically_increasing_id(), spark_partition_id(), + rand(Random.nextLong()), randn(Random.nextLong()) + ).foreach(assertValuesDoNotChangeAfterCoalesceOrUnion(_)) + } +} + +object DataFrameFunctionsSuite { + case class CodegenFallbackExpr(child: Expression) extends Expression with CodegenFallback { + override def children: Seq[Expression] = Seq(child) + override def nullable: Boolean = child.nullable + override def dataType: DataType = child.dataType + override lazy val resolved = true + override def eval(input: InternalRow): Any = child.eval(input) + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala index 231f204b12b4..c80695bd3e0f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala @@ -154,8 +154,9 @@ case class HiveTableScanExec( val numOutputRows = longMetric("numOutputRows") // Avoid to serialize MetastoreRelation because schema is lazy. (see SPARK-15649) val outputSchema = schema - rdd.mapPartitionsInternal { iter => + rdd.mapPartitionsWithIndexInternal { (index, iter) => val proj = UnsafeProjection.create(outputSchema) + proj.initialize(index) iter.map { r => numOutputRows += 1 proj(r) From 3c24299b71e23e159edbb972347b13430f92a465 Mon Sep 17 00:00:00 2001 From: Jeff Zhang Date: Wed, 2 Nov 2016 11:47:45 -0700 Subject: [PATCH 051/132] [SPARK-18160][CORE][YARN] spark.files & spark.jars should not be passed to driver in yarn mode ## What changes were proposed in this pull request? spark.files is still passed to driver in yarn mode, so SparkContext will still handle it which cause the error in the jira desc. ## How was this patch tested? Tested manually in a 5 node cluster. As this issue only happens in multiple node cluster, so I didn't write test for it. Author: Jeff Zhang Closes #15669 from zjffdu/SPARK-18160. --- .../scala/org/apache/spark/SparkContext.scala | 29 ++++--------------- .../org/apache/spark/deploy/yarn/Client.scala | 5 +++- 2 files changed, 10 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 4694790c72cd..63478c88b057 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1716,29 +1716,12 @@ class SparkContext(config: SparkConf) extends Logging { key = uri.getScheme match { // A JAR file which exists only on the driver node case null | "file" => - if (master == "yarn" && deployMode == "cluster") { - // In order for this to work in yarn cluster mode the user must specify the - // --addJars option to the client to upload the file into the distributed cache - // of the AM to make it show up in the current working directory. - val fileName = new Path(uri.getPath).getName() - try { - env.rpcEnv.fileServer.addJar(new File(fileName)) - } catch { - case e: Exception => - // For now just log an error but allow to go through so spark examples work. - // The spark examples don't really need the jar distributed since its also - // the app jar. - logError("Error adding jar (" + e + "), was the --addJars option used?") - null - } - } else { - try { - env.rpcEnv.fileServer.addJar(new File(uri.getPath)) - } catch { - case exc: FileNotFoundException => - logError(s"Jar not found at $path") - null - } + try { + env.rpcEnv.fileServer.addJar(new File(uri.getPath)) + } catch { + case exc: FileNotFoundException => + logError(s"Jar not found at $path") + null } // A JAR file which exists locally on every worker node case "local" => diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 55e4a833b670..053a78617d4e 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -1202,7 +1202,10 @@ private object Client extends Logging { // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes System.setProperty("SPARK_YARN_MODE", "true") val sparkConf = new SparkConf - + // SparkSubmit would use yarn cache to distribute files & jars in yarn mode, + // so remove them from sparkConf here for yarn mode. + sparkConf.remove("spark.jars") + sparkConf.remove("spark.files") val args = new ClientArguments(argStrings) new Client(args, sparkConf).run() } From 37d95227a21de602b939dae84943ba007f434513 Mon Sep 17 00:00:00 2001 From: Steve Loughran Date: Wed, 2 Nov 2016 11:52:29 -0700 Subject: [PATCH 052/132] [SPARK-17058][BUILD] Add maven snapshots-and-staging profile to build/test against staging artifacts ## What changes were proposed in this pull request? Adds a `snapshots-and-staging profile` so that RCs of projects like Hadoop and HBase can be used in developer-only build and test runs. There's a comment above the profile telling people not to use this in production. There's no attempt to do the same for SBT, as Ivy is different. ## How was this patch tested? Tested by building against the Hadoop 2.7.3 RC 1 JARs without the profile (and without any local copy of the 2.7.3 artifacts), the build failed ``` mvn install -DskipTests -Pyarn,hadoop-2.7,hive -Dhadoop.version=2.7.3 ... [INFO] ------------------------------------------------------------------------ [INFO] Building Spark Project Launcher 2.1.0-SNAPSHOT [INFO] ------------------------------------------------------------------------ Downloading: https://repo1.maven.org/maven2/org/apache/hadoop/hadoop-client/2.7.3/hadoop-client-2.7.3.pom [WARNING] The POM for org.apache.hadoop:hadoop-client:jar:2.7.3 is missing, no dependency information available Downloading: https://repo1.maven.org/maven2/org/apache/hadoop/hadoop-client/2.7.3/hadoop-client-2.7.3.jar [INFO] ------------------------------------------------------------------------ [INFO] Reactor Summary: [INFO] [INFO] Spark Project Parent POM ........................... SUCCESS [ 4.482 s] [INFO] Spark Project Tags ................................. SUCCESS [ 17.402 s] [INFO] Spark Project Sketch ............................... SUCCESS [ 11.252 s] [INFO] Spark Project Networking ........................... SUCCESS [ 13.458 s] [INFO] Spark Project Shuffle Streaming Service ............ SUCCESS [ 9.043 s] [INFO] Spark Project Unsafe ............................... SUCCESS [ 16.027 s] [INFO] Spark Project Launcher ............................. FAILURE [ 1.653 s] [INFO] Spark Project Core ................................. SKIPPED ... ``` With the profile, the build completed ``` mvn install -DskipTests -Pyarn,hadoop-2.7,hive,snapshots-and-staging -Dhadoop.version=2.7.3 ``` Author: Steve Loughran Closes #14646 from steveloughran/stevel/SPARK-17058-support-asf-snapshots. --- pom.xml | 48 ++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 48 insertions(+) diff --git a/pom.xml b/pom.xml index aaf7cfa7eb2a..04d2eaa1d3ba 100644 --- a/pom.xml +++ b/pom.xml @@ -2693,6 +2693,54 @@ + + + snapshots-and-staging + + + https://repository.apache.org/content/groups/staging/ + https://repository.apache.org/content/repositories/snapshots/ + + + + + ASF Staging + ${asf.staging} + + + ASF Snapshots + ${asf.snapshots} + + true + + + false + + + + + + + ASF Staging + ${asf.staging} + + + ASF Snapshots + ${asf.snapshots} + + true + + + false + + + + +