From 99b7187c2dce4c73829b9b32de80b02a053763cc Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 11 Mar 2016 10:27:38 -0800 Subject: [PATCH 01/21] [SPARK-13780][SQL] Add missing dependency to build. This is needed to avoid odd compiler errors when building just the sql package with maven, because of odd interactions between scalac and shaded classes. Author: Marcelo Vanzin Closes #11640 from vanzin/SPARK-13780. --- sql/core/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 89e01fc01596e..708670b2923fc 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -83,6 +83,10 @@ org.apache.parquet parquet-hadoop + + org.eclipse.jetty + jetty-servlet + com.fasterxml.jackson.core jackson-databind From eb650a81f14fa7bc665856397e19ddf1a92ca3c5 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Fri, 11 Mar 2016 11:07:27 -0800 Subject: [PATCH 02/21] [STREAMING][MINOR] Fix a duplicate "be" in comments Author: Liwei Lin Closes #11650 from lw-lin/typo. --- .../src/main/scala/org/apache/spark/streaming/StateSpec.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StateSpec.scala b/streaming/src/main/scala/org/apache/spark/streaming/StateSpec.scala index e6724feaee105..7c1ea2f89ddb8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StateSpec.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StateSpec.scala @@ -83,8 +83,7 @@ sealed abstract class StateSpec[KeyType, ValueType, StateType, MappedType] exten def numPartitions(numPartitions: Int): this.type /** - * Set the partitioner by which the state RDDs generated by `mapWithState` will be - * be partitioned. + * Set the partitioner by which the state RDDs generated by `mapWithState` will be partitioned. */ def partitioner(partitioner: Partitioner): this.type From ff776b2fc1cd4c571fd542dbf807e6fa3373cb34 Mon Sep 17 00:00:00 2001 From: Nezih Yigitbasi Date: Fri, 11 Mar 2016 11:11:53 -0800 Subject: [PATCH 03/21] [SPARK-13328][CORE] Poor read performance for broadcast variables with dynamic resource allocation When dynamic resource allocation is enabled fetching broadcast variables from removed executors were causing job failures and SPARK-9591 fixed this problem by trying all locations of a block before giving up. However, the locations of a block is retrieved only once from the driver in this process and the locations in this list can be stale due to dynamic resource allocation. This situation gets worse when running on a large cluster as the size of this location list can be in the order of several hundreds out of which there may be tens of stale entries. What we have observed is with the default settings of 3 max retries and 5s between retries (that's 15s per location) the time it takes to read a broadcast variable can be as high as ~17m (70 failed attempts * 15s/attempt) Author: Nezih Yigitbasi Closes #11241 from nezihyigitbasi/SPARK-13328. --- .../apache/spark/storage/BlockManager.scala | 47 ++++++++--- .../spark/storage/BlockManagerSuite.scala | 84 ++++++++++++++++++- 2 files changed, 116 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 873330e136e22..bcf65e9d7e25c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -133,6 +133,9 @@ private[spark] class BlockManager( private val compressRdds = conf.getBoolean("spark.rdd.compress", false) // Whether to compress shuffle output temporarily spilled to disk private val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", true) + // Max number of failures before this block manager refreshes the block locations from the driver + private val maxFailuresBeforeLocationRefresh = + conf.getInt("spark.block.failures.beforeLocationRefresh", 5) private val slaveEndpoint = rpcEnv.setupEndpoint( "BlockManagerEndpoint" + BlockManager.ID_GENERATOR.next, @@ -568,26 +571,46 @@ private[spark] class BlockManager( def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { logDebug(s"Getting remote block $blockId") require(blockId != null, "BlockId is null") + var runningFailureCount = 0 + var totalFailureCount = 0 val locations = getLocations(blockId) - var numFetchFailures = 0 - for (loc <- locations) { + val maxFetchFailures = locations.size + var locationIterator = locations.iterator + while (locationIterator.hasNext) { + val loc = locationIterator.next() logDebug(s"Getting remote block $blockId from $loc") val data = try { blockTransferService.fetchBlockSync( loc.host, loc.port, loc.executorId, blockId.toString).nioByteBuffer() } catch { case NonFatal(e) => - numFetchFailures += 1 - if (numFetchFailures == locations.size) { - // An exception is thrown while fetching this block from all locations - throw new BlockFetchException(s"Failed to fetch block from" + - s" ${locations.size} locations. Most recent failure cause:", e) - } else { - // This location failed, so we retry fetch from a different one by returning null here - logWarning(s"Failed to fetch remote block $blockId " + - s"from $loc (failed attempt $numFetchFailures)", e) - null + runningFailureCount += 1 + totalFailureCount += 1 + + if (totalFailureCount >= maxFetchFailures) { + // Give up trying anymore locations. Either we've tried all of the original locations, + // or we've refreshed the list of locations from the master, and have still + // hit failures after trying locations from the refreshed list. + throw new BlockFetchException(s"Failed to fetch block after" + + s" ${totalFailureCount} fetch failures. Most recent failure cause:", e) + } + + logWarning(s"Failed to fetch remote block $blockId " + + s"from $loc (failed attempt $runningFailureCount)", e) + + // If there is a large number of executors then locations list can contain a + // large number of stale entries causing a large number of retries that may + // take a significant amount of time. To get rid of these stale entries + // we refresh the block locations after a certain number of fetch failures + if (runningFailureCount >= maxFailuresBeforeLocationRefresh) { + locationIterator = getLocations(blockId).iterator + logDebug(s"Refreshed locations from the driver " + + s"after ${runningFailureCount} fetch failures.") + runningFailureCount = 0 } + + // This location failed, so we retry fetch from a different one by returning null here + null } if (data != null) { diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 42595c8cf2daa..dc4be1467794c 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -21,11 +21,12 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ +import scala.concurrent.Future import scala.language.implicitConversions import scala.language.postfixOps import org.mockito.{Matchers => mc} -import org.mockito.Mockito.{mock, when} +import org.mockito.Mockito.{mock, times, verify, when} import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ @@ -33,7 +34,10 @@ import org.scalatest.concurrent.Timeouts._ import org.apache.spark._ import org.apache.spark.executor.DataReadMethod import org.apache.spark.memory.StaticMemoryManager +import org.apache.spark.network.{BlockDataManager, BlockTransferService} +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.netty.NettyBlockTransferService +import org.apache.spark.network.shuffle.BlockFetchingListener import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} @@ -66,9 +70,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE private def makeBlockManager( maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER, - master: BlockManagerMaster = this.master): BlockManager = { + master: BlockManagerMaster = this.master, + transferService: Option[BlockTransferService] = Option.empty): BlockManager = { val serializer = new KryoSerializer(conf) - val transfer = new NettyBlockTransferService(conf, securityMgr, numCores = 1) + val transfer = transferService + .getOrElse(new NettyBlockTransferService(conf, securityMgr, numCores = 1)) val memManager = new StaticMemoryManager(conf, Long.MaxValue, maxMem, numCores = 1) val blockManager = new BlockManager(name, rpcEnv, master, serializer, conf, memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) @@ -1287,6 +1293,78 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(store.getSingle("a1").isDefined, "a1 was not in store") assert(store.getSingle("a3").isDefined, "a3 was not in store") } + + test("SPARK-13328: refresh block locations (fetch should fail after hitting a threshold)") { + val mockBlockTransferService = + new MockBlockTransferService(conf.getInt("spark.block.failures.beforeLocationRefresh", 5)) + store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) + store.putSingle("item", 999L, StorageLevel.MEMORY_ONLY, tellMaster = true) + intercept[BlockFetchException] { + store.getRemoteBytes("item") + } + } + + test("SPARK-13328: refresh block locations (fetch should succeed after location refresh)") { + val maxFailuresBeforeLocationRefresh = + conf.getInt("spark.block.failures.beforeLocationRefresh", 5) + val mockBlockManagerMaster = mock(classOf[BlockManagerMaster]) + val mockBlockTransferService = + new MockBlockTransferService(maxFailuresBeforeLocationRefresh) + // make sure we have more than maxFailuresBeforeLocationRefresh locations + // so that we have a chance to do location refresh + val blockManagerIds = (0 to maxFailuresBeforeLocationRefresh) + .map { i => BlockManagerId(s"id-$i", s"host-$i", i + 1) } + when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn(blockManagerIds) + store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, + transferService = Option(mockBlockTransferService)) + val block = store.getRemoteBytes("item") + .asInstanceOf[Option[ByteBuffer]] + assert(block.isDefined) + verify(mockBlockManagerMaster, times(2)).getLocations("item") + } + + class MockBlockTransferService(val maxFailures: Int) extends BlockTransferService { + var numCalls = 0 + + override def init(blockDataManager: BlockDataManager): Unit = {} + + override def fetchBlocks( + host: String, + port: Int, + execId: String, + blockIds: Array[String], + listener: BlockFetchingListener): Unit = { + listener.onBlockFetchSuccess("mockBlockId", new NioManagedBuffer(ByteBuffer.allocate(1))) + } + + override def close(): Unit = {} + + override def hostName: String = { "MockBlockTransferServiceHost" } + + override def port: Int = { 63332 } + + override def uploadBlock( + hostname: String, + port: Int, execId: String, + blockId: BlockId, + blockData: ManagedBuffer, + level: StorageLevel): Future[Unit] = { + import scala.concurrent.ExecutionContext.Implicits.global + Future {} + } + + override def fetchBlockSync( + host: String, + port: Int, + execId: String, + blockId: String): ManagedBuffer = { + numCalls += 1 + if (numCalls <= maxFailures) { + throw new RuntimeException("Failing block fetch in the mock block transfer service") + } + super.fetchBlockSync(host, port, execId, blockId) + } + } } private object BlockManagerSuite { From 073bf9d4d91e0242a813f3d227e52e76c26a2200 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 11 Mar 2016 11:18:51 -0800 Subject: [PATCH 04/21] [SPARK-13807] De-duplicate `Python*Helper` instantiation code in PySpark streaming This patch de-duplicates code in PySpark streaming which loads the `Python*Helper` classes. I also changed a few `raise e` statements to simply `raise` in order to preserve the full exception stacktrace when re-throwing. Here's a link to the whitespace-change-free diff: https://github.com/apache/spark/compare/master...JoshRosen:pyspark-reflection-deduplication?w=0 Author: Josh Rosen Closes #11641 from JoshRosen/pyspark-reflection-deduplication. --- python/pyspark/streaming/flume.py | 40 +++++------ python/pyspark/streaming/kafka.py | 100 ++++++++++++---------------- python/pyspark/streaming/kinesis.py | 2 +- python/pyspark/streaming/mqtt.py | 2 +- 4 files changed, 60 insertions(+), 84 deletions(-) diff --git a/python/pyspark/streaming/flume.py b/python/pyspark/streaming/flume.py index b1fff0a5c7d6b..edd5886a85079 100644 --- a/python/pyspark/streaming/flume.py +++ b/python/pyspark/streaming/flume.py @@ -55,17 +55,8 @@ def createStream(ssc, hostname, port, :return: A DStream object """ jlevel = ssc._sc._getJavaStorageLevel(storageLevel) - - try: - helperClass = ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader()\ - .loadClass("org.apache.spark.streaming.flume.FlumeUtilsPythonHelper") - helper = helperClass.newInstance() - jstream = helper.createStream(ssc._jssc, hostname, port, jlevel, enableDecompression) - except Py4JJavaError as e: - if 'ClassNotFoundException' in str(e.java_exception): - FlumeUtils._printErrorMsg(ssc.sparkContext) - raise e - + helper = FlumeUtils._get_helper(ssc._sc) + jstream = helper.createStream(ssc._jssc, hostname, port, jlevel, enableDecompression) return FlumeUtils._toPythonDStream(ssc, jstream, bodyDecoder) @staticmethod @@ -95,18 +86,9 @@ def createPollingStream(ssc, addresses, for (host, port) in addresses: hosts.append(host) ports.append(port) - - try: - helperClass = ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \ - .loadClass("org.apache.spark.streaming.flume.FlumeUtilsPythonHelper") - helper = helperClass.newInstance() - jstream = helper.createPollingStream( - ssc._jssc, hosts, ports, jlevel, maxBatchSize, parallelism) - except Py4JJavaError as e: - if 'ClassNotFoundException' in str(e.java_exception): - FlumeUtils._printErrorMsg(ssc.sparkContext) - raise e - + helper = FlumeUtils._get_helper(ssc._sc) + jstream = helper.createPollingStream( + ssc._jssc, hosts, ports, jlevel, maxBatchSize, parallelism) return FlumeUtils._toPythonDStream(ssc, jstream, bodyDecoder) @staticmethod @@ -126,6 +108,18 @@ def func(event): return (headers, body) return stream.map(func) + @staticmethod + def _get_helper(sc): + try: + helperClass = sc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \ + .loadClass("org.apache.spark.streaming.flume.FlumeUtilsPythonHelper") + return helperClass.newInstance() + except Py4JJavaError as e: + # TODO: use --jar once it also work on driver + if 'ClassNotFoundException' in str(e.java_exception): + FlumeUtils._printErrorMsg(sc) + raise + @staticmethod def _printErrorMsg(sc): print(""" diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py index 13f8f9578e62a..a70b99249d3a2 100644 --- a/python/pyspark/streaming/kafka.py +++ b/python/pyspark/streaming/kafka.py @@ -66,18 +66,8 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams=None, if not isinstance(topics, dict): raise TypeError("topics should be dict") jlevel = ssc._sc._getJavaStorageLevel(storageLevel) - - try: - # Use KafkaUtilsPythonHelper to access Scala's KafkaUtils (see SPARK-6027) - helperClass = ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader()\ - .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper") - helper = helperClass.newInstance() - jstream = helper.createStream(ssc._jssc, kafkaParams, topics, jlevel) - except Py4JJavaError as e: - # TODO: use --jar once it also work on driver - if 'ClassNotFoundException' in str(e.java_exception): - KafkaUtils._printErrorMsg(ssc.sparkContext) - raise e + helper = KafkaUtils._get_helper(ssc._sc) + jstream = helper.createStream(ssc._jssc, kafkaParams, topics, jlevel) ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) stream = DStream(jstream, ssc, ser) return stream.map(lambda k_v: (keyDecoder(k_v[0]), valueDecoder(k_v[1]))) @@ -129,27 +119,20 @@ def funcWithMessageHandler(m): m._set_value_decoder(valueDecoder) return messageHandler(m) - try: - helperClass = ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \ - .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper") - helper = helperClass.newInstance() - - jfromOffsets = dict([(k._jTopicAndPartition(helper), - v) for (k, v) in fromOffsets.items()]) - if messageHandler is None: - ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) - func = funcWithoutMessageHandler - jstream = helper.createDirectStreamWithoutMessageHandler( - ssc._jssc, kafkaParams, set(topics), jfromOffsets) - else: - ser = AutoBatchedSerializer(PickleSerializer()) - func = funcWithMessageHandler - jstream = helper.createDirectStreamWithMessageHandler( - ssc._jssc, kafkaParams, set(topics), jfromOffsets) - except Py4JJavaError as e: - if 'ClassNotFoundException' in str(e.java_exception): - KafkaUtils._printErrorMsg(ssc.sparkContext) - raise e + helper = KafkaUtils._get_helper(ssc._sc) + + jfromOffsets = dict([(k._jTopicAndPartition(helper), + v) for (k, v) in fromOffsets.items()]) + if messageHandler is None: + ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) + func = funcWithoutMessageHandler + jstream = helper.createDirectStreamWithoutMessageHandler( + ssc._jssc, kafkaParams, set(topics), jfromOffsets) + else: + ser = AutoBatchedSerializer(PickleSerializer()) + func = funcWithMessageHandler + jstream = helper.createDirectStreamWithMessageHandler( + ssc._jssc, kafkaParams, set(topics), jfromOffsets) stream = DStream(jstream, ssc, ser).map(func) return KafkaDStream(stream._jdstream, ssc, stream._jrdd_deserializer) @@ -189,28 +172,35 @@ def funcWithMessageHandler(m): m._set_value_decoder(valueDecoder) return messageHandler(m) + helper = KafkaUtils._get_helper(sc) + + joffsetRanges = [o._jOffsetRange(helper) for o in offsetRanges] + jleaders = dict([(k._jTopicAndPartition(helper), + v._jBroker(helper)) for (k, v) in leaders.items()]) + if messageHandler is None: + jrdd = helper.createRDDWithoutMessageHandler( + sc._jsc, kafkaParams, joffsetRanges, jleaders) + ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) + rdd = RDD(jrdd, sc, ser).map(funcWithoutMessageHandler) + else: + jrdd = helper.createRDDWithMessageHandler( + sc._jsc, kafkaParams, joffsetRanges, jleaders) + rdd = RDD(jrdd, sc).map(funcWithMessageHandler) + + return KafkaRDD(rdd._jrdd, sc, rdd._jrdd_deserializer) + + @staticmethod + def _get_helper(sc): try: + # Use KafkaUtilsPythonHelper to access Scala's KafkaUtils (see SPARK-6027) helperClass = sc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \ .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper") - helper = helperClass.newInstance() - joffsetRanges = [o._jOffsetRange(helper) for o in offsetRanges] - jleaders = dict([(k._jTopicAndPartition(helper), - v._jBroker(helper)) for (k, v) in leaders.items()]) - if messageHandler is None: - jrdd = helper.createRDDWithoutMessageHandler( - sc._jsc, kafkaParams, joffsetRanges, jleaders) - ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) - rdd = RDD(jrdd, sc, ser).map(funcWithoutMessageHandler) - else: - jrdd = helper.createRDDWithMessageHandler( - sc._jsc, kafkaParams, joffsetRanges, jleaders) - rdd = RDD(jrdd, sc).map(funcWithMessageHandler) + return helperClass.newInstance() except Py4JJavaError as e: + # TODO: use --jar once it also work on driver if 'ClassNotFoundException' in str(e.java_exception): KafkaUtils._printErrorMsg(sc) - raise e - - return KafkaRDD(rdd._jrdd, sc, rdd._jrdd_deserializer) + raise @staticmethod def _printErrorMsg(sc): @@ -333,16 +323,8 @@ def offsetRanges(self): Get the OffsetRange of specific KafkaRDD. :return: A list of OffsetRange """ - try: - helperClass = self.ctx._jvm.java.lang.Thread.currentThread().getContextClassLoader() \ - .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper") - helper = helperClass.newInstance() - joffsetRanges = helper.offsetRangesOfKafkaRDD(self._jrdd.rdd()) - except Py4JJavaError as e: - if 'ClassNotFoundException' in str(e.java_exception): - KafkaUtils._printErrorMsg(self.ctx) - raise e - + helper = KafkaUtils._get_helper(self.ctx) + joffsetRanges = helper.offsetRangesOfKafkaRDD(self._jrdd.rdd()) ranges = [OffsetRange(o.topic(), o.partition(), o.fromOffset(), o.untilOffset()) for o in joffsetRanges] return ranges diff --git a/python/pyspark/streaming/kinesis.py b/python/pyspark/streaming/kinesis.py index af72c3d6903f9..e681301681a81 100644 --- a/python/pyspark/streaming/kinesis.py +++ b/python/pyspark/streaming/kinesis.py @@ -83,7 +83,7 @@ def createStream(ssc, kinesisAppName, streamName, endpointUrl, regionName, except Py4JJavaError as e: if 'ClassNotFoundException' in str(e.java_exception): KinesisUtils._printErrorMsg(ssc.sparkContext) - raise e + raise stream = DStream(jstream, ssc, NoOpSerializer()) return stream.map(lambda v: decoder(v)) diff --git a/python/pyspark/streaming/mqtt.py b/python/pyspark/streaming/mqtt.py index 3a515ea4996f4..388e9526ba73a 100644 --- a/python/pyspark/streaming/mqtt.py +++ b/python/pyspark/streaming/mqtt.py @@ -48,7 +48,7 @@ def createStream(ssc, brokerUrl, topic, except Py4JJavaError as e: if 'ClassNotFoundException' in str(e.java_exception): MQTTUtils._printErrorMsg(ssc.sparkContext) - raise e + raise return DStream(jstream, ssc, UTF8Deserializer()) From 42afd72c654318e9fb1f2a204198221e797c2485 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Fri, 11 Mar 2016 13:49:37 -0800 Subject: [PATCH 05/21] [SPARK-13814] [PYSPARK] Delete unnecessary imports in python examples files JIRA: https://issues.apache.org/jira/browse/SPARK-13814 ## What changes were proposed in this pull request? delete unnecessary imports in python examples files ## How was this patch tested? manual tests Author: Zheng RuiFeng Closes #11651 from zhengruifeng/del_import_pe. --- examples/src/main/python/ml/als_example.py | 2 -- .../src/main/python/ml/decision_tree_classification_example.py | 2 -- examples/src/main/python/ml/decision_tree_regression_example.py | 2 -- .../main/python/ml/gradient_boosted_tree_classifier_example.py | 2 -- .../main/python/ml/gradient_boosted_tree_regressor_example.py | 2 -- examples/src/main/python/ml/kmeans_example.py | 1 - examples/src/main/python/ml/random_forest_classifier_example.py | 2 -- examples/src/main/python/ml/random_forest_regressor_example.py | 2 -- .../main/python/mllib/binary_classification_metrics_example.py | 1 - .../python/mllib/gradient_boosting_classification_example.py | 2 -- .../main/python/mllib/gradient_boosting_regression_example.py | 2 -- .../main/python/mllib/random_forest_classification_example.py | 2 -- .../src/main/python/mllib/random_forest_regression_example.py | 2 -- examples/src/main/python/mllib/recommendation_example.py | 2 -- examples/src/main/python/streaming/sql_network_wordcount.py | 1 - 15 files changed, 27 deletions(-) diff --git a/examples/src/main/python/ml/als_example.py b/examples/src/main/python/ml/als_example.py index f61c8ab5d6328..922173308c6aa 100644 --- a/examples/src/main/python/ml/als_example.py +++ b/examples/src/main/python/ml/als_example.py @@ -21,8 +21,6 @@ from pyspark.sql import SQLContext # $example on$ -import math - from pyspark.ml.evaluation import RegressionEvaluator from pyspark.ml.recommendation import ALS from pyspark.sql import Row diff --git a/examples/src/main/python/ml/decision_tree_classification_example.py b/examples/src/main/python/ml/decision_tree_classification_example.py index 8cda56dbb9bdf..86bdc65392bbb 100644 --- a/examples/src/main/python/ml/decision_tree_classification_example.py +++ b/examples/src/main/python/ml/decision_tree_classification_example.py @@ -20,8 +20,6 @@ """ from __future__ import print_function -import sys - # $example on$ from pyspark import SparkContext, SQLContext from pyspark.ml import Pipeline diff --git a/examples/src/main/python/ml/decision_tree_regression_example.py b/examples/src/main/python/ml/decision_tree_regression_example.py index 439e398947499..8e20d5d8572a5 100644 --- a/examples/src/main/python/ml/decision_tree_regression_example.py +++ b/examples/src/main/python/ml/decision_tree_regression_example.py @@ -20,8 +20,6 @@ """ from __future__ import print_function -import sys - from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline diff --git a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py index 028497651fbf9..f7e842f4b303a 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_classifier_example.py @@ -20,8 +20,6 @@ """ from __future__ import print_function -import sys - from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline diff --git a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py index 4246e133a9030..f8b4de651c768 100644 --- a/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py +++ b/examples/src/main/python/ml/gradient_boosted_tree_regressor_example.py @@ -20,8 +20,6 @@ """ from __future__ import print_function -import sys - from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline diff --git a/examples/src/main/python/ml/kmeans_example.py b/examples/src/main/python/ml/kmeans_example.py index 150dadd42f33e..fa57a4d3ada1b 100644 --- a/examples/src/main/python/ml/kmeans_example.py +++ b/examples/src/main/python/ml/kmeans_example.py @@ -18,7 +18,6 @@ from __future__ import print_function import sys -import re import numpy as np from pyspark import SparkContext diff --git a/examples/src/main/python/ml/random_forest_classifier_example.py b/examples/src/main/python/ml/random_forest_classifier_example.py index b3530d4f41c8e..c3570438c51d9 100644 --- a/examples/src/main/python/ml/random_forest_classifier_example.py +++ b/examples/src/main/python/ml/random_forest_classifier_example.py @@ -20,8 +20,6 @@ """ from __future__ import print_function -import sys - from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline diff --git a/examples/src/main/python/ml/random_forest_regressor_example.py b/examples/src/main/python/ml/random_forest_regressor_example.py index b59c7c9414841..b77014f379237 100644 --- a/examples/src/main/python/ml/random_forest_regressor_example.py +++ b/examples/src/main/python/ml/random_forest_regressor_example.py @@ -20,8 +20,6 @@ """ from __future__ import print_function -import sys - from pyspark import SparkContext, SQLContext # $example on$ from pyspark.ml import Pipeline diff --git a/examples/src/main/python/mllib/binary_classification_metrics_example.py b/examples/src/main/python/mllib/binary_classification_metrics_example.py index 437acb998acc3..4e7ea289b2532 100644 --- a/examples/src/main/python/mllib/binary_classification_metrics_example.py +++ b/examples/src/main/python/mllib/binary_classification_metrics_example.py @@ -18,7 +18,6 @@ Binary Classification Metrics Example. """ from __future__ import print_function -import sys from pyspark import SparkContext, SQLContext # $example on$ from pyspark.mllib.classification import LogisticRegressionWithLBFGS diff --git a/examples/src/main/python/mllib/gradient_boosting_classification_example.py b/examples/src/main/python/mllib/gradient_boosting_classification_example.py index a94ea0d582e59..b204cd1b31c86 100644 --- a/examples/src/main/python/mllib/gradient_boosting_classification_example.py +++ b/examples/src/main/python/mllib/gradient_boosting_classification_example.py @@ -20,8 +20,6 @@ """ from __future__ import print_function -import sys - from pyspark import SparkContext # $example on$ from pyspark.mllib.tree import GradientBoostedTrees, GradientBoostedTreesModel diff --git a/examples/src/main/python/mllib/gradient_boosting_regression_example.py b/examples/src/main/python/mllib/gradient_boosting_regression_example.py index 86040799dc1d9..758e224a9e21d 100644 --- a/examples/src/main/python/mllib/gradient_boosting_regression_example.py +++ b/examples/src/main/python/mllib/gradient_boosting_regression_example.py @@ -20,8 +20,6 @@ """ from __future__ import print_function -import sys - from pyspark import SparkContext # $example on$ from pyspark.mllib.tree import GradientBoostedTrees, GradientBoostedTreesModel diff --git a/examples/src/main/python/mllib/random_forest_classification_example.py b/examples/src/main/python/mllib/random_forest_classification_example.py index 324ba50625d25..9e5a8dcaabb0e 100644 --- a/examples/src/main/python/mllib/random_forest_classification_example.py +++ b/examples/src/main/python/mllib/random_forest_classification_example.py @@ -20,8 +20,6 @@ """ from __future__ import print_function -import sys - from pyspark import SparkContext # $example on$ from pyspark.mllib.tree import RandomForest, RandomForestModel diff --git a/examples/src/main/python/mllib/random_forest_regression_example.py b/examples/src/main/python/mllib/random_forest_regression_example.py index f7aa6114eceb3..2e1be34c1a29a 100644 --- a/examples/src/main/python/mllib/random_forest_regression_example.py +++ b/examples/src/main/python/mllib/random_forest_regression_example.py @@ -20,8 +20,6 @@ """ from __future__ import print_function -import sys - from pyspark import SparkContext # $example on$ from pyspark.mllib.tree import RandomForest, RandomForestModel diff --git a/examples/src/main/python/mllib/recommendation_example.py b/examples/src/main/python/mllib/recommendation_example.py index 615db0749b182..00e683c3ae938 100644 --- a/examples/src/main/python/mllib/recommendation_example.py +++ b/examples/src/main/python/mllib/recommendation_example.py @@ -20,8 +20,6 @@ """ from __future__ import print_function -import sys - from pyspark import SparkContext # $example on$ diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py index da90c07dbd82f..1ba5e9fb78993 100644 --- a/examples/src/main/python/streaming/sql_network_wordcount.py +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -29,7 +29,6 @@ """ from __future__ import print_function -import os import sys from pyspark import SparkContext From 66d9d0edfef986895490bcdeacbc0ca38e091702 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 11 Mar 2016 15:13:48 -0800 Subject: [PATCH 06/21] [SPARK-13139][SQL] Parse Hive DDL commands ourselves ## What changes were proposed in this pull request? This patch is ported over from viirya's changes in #11048. Currently for most DDLs we just pass the query text directly to Hive. Instead, we should parse these commands ourselves and in the future (not part of this patch) use the `HiveCatalog` to process these DDLs. This is a pretext to merging `SQLContext` and `HiveContext`. Note: As of this patch we still pass the query text to Hive. The difference is that we now parse the commands ourselves so in the future we can just use our own catalog. ## How was this patch tested? Jenkins, new `DDLCommandSuite`, which comprises of about 40% of the changes here. Author: Andrew Or Closes #11573 from andrewor14/parser-plus-plus. --- .../sql/catalyst/parser/ParserUtils.scala | 11 +- .../org/apache/spark/sql/SQLContext.scala | 9 + .../apache/spark/sql/execution/SparkQl.scala | 125 +++- .../command/AlterTableCommandParser.scala | 428 ++++++++++++++ .../spark/sql/execution/command/ddl.scala | 210 +++++++ .../execution/command/DDLCommandSuite.scala | 544 ++++++++++++++++++ .../apache/spark/sql/hive/HiveContext.scala | 9 + .../org/apache/spark/sql/hive/HiveQl.scala | 18 +- 8 files changed, 1318 insertions(+), 36 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala index c105b53f1fc62..0c2e481954a5e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserUtils.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.parser +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.types._ @@ -29,6 +30,7 @@ import org.apache.spark.sql.types._ object ParserUtils { object Token { + // Match on (text, children) def unapply(node: ASTNode): Some[(String, List[ASTNode])] = { CurrentOrigin.setPosition(node.line, node.positionInLine) node.pattern @@ -160,7 +162,14 @@ object ParserUtils { } /** - * Throw an exception because we cannot parse the given node. + * Throw an exception because we cannot parse the given node for some unexpected reason. + */ + def parseFailed(msg: String, node: ASTNode): Nothing = { + throw new AnalysisException(s"$msg: '${node.source}") + } + + /** + * Throw an exception because there are no rules to parse the node. */ def noParseRule(msg: String, node: ASTNode): Nothing = { throw new NotImplementedError( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 49a70a7c5f200..36fe57f78be1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -784,6 +784,15 @@ class SQLContext private[sql]( Dataset.newDataFrame(this, parseSql(sqlText)) } + /** + * Executes a SQL query without parsing it, but instead passing it directly to an underlying + * system to process. This is currently only used for Hive DDLs and will be removed as soon + * as Spark can parse all supported Hive DDLs itself. + */ + private[sql] def runNativeSql(sqlText: String): Seq[Row] = { + throw new UnsupportedOperationException + } + /** * Returns the specified table as a [[DataFrame]]. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala index 471a5e436c8ee..d12dab567b00a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala @@ -29,7 +29,26 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly import ParserUtils._ /** Check if a command should not be explained. */ - protected def isNoExplainCommand(command: String): Boolean = "TOK_DESCTABLE" == command + protected def isNoExplainCommand(command: String): Boolean = { + "TOK_DESCTABLE" == command || "TOK_ALTERTABLE" == command + } + + /** + * For each node, extract properties in the form of a list ['key1', 'key2', 'key3', 'value'] + * into a pair (key1.key2.key3, value). + */ + private def extractProps( + props: Seq[ASTNode], + expectedNodeText: String): Seq[(String, String)] = { + props.map { + case Token(x, keysAndValue) if x == expectedNodeText => + val key = keysAndValue.init.map { x => unquoteString(x.text) }.mkString(".") + val value = unquoteString(keysAndValue.last.text) + (key, value) + case p => + parseFailed(s"Expected property '$expectedNodeText' in command", p) + } + } protected override def nodeToPlan(node: ASTNode): LogicalPlan = { node match { @@ -64,10 +83,86 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly val tableIdent = extractTableIdent(nameParts) RefreshTable(tableIdent) + // CREATE DATABASE [IF NOT EXISTS] database_name [COMMENT database_comment] + // [LOCATION path] [WITH DBPROPERTIES (key1=val1, key2=val2, ...)]; + case Token("TOK_CREATEDATABASE", Token(databaseName, Nil) :: args) => + val Seq(ifNotExists, dbLocation, databaseComment, dbprops) = getClauses(Seq( + "TOK_IFNOTEXISTS", + "TOK_DATABASELOCATION", + "TOK_DATABASECOMMENT", + "TOK_DATABASEPROPERTIES"), args) + val location = dbLocation.map { + case Token("TOK_DATABASELOCATION", Token(loc, Nil) :: Nil) => unquoteString(loc) + case _ => parseFailed("Invalid CREATE DATABASE command", node) + } + val comment = databaseComment.map { + case Token("TOK_DATABASECOMMENT", Token(com, Nil) :: Nil) => unquoteString(com) + case _ => parseFailed("Invalid CREATE DATABASE command", node) + } + val props = dbprops.toSeq.flatMap { + case Token("TOK_DATABASEPROPERTIES", Token("TOK_DBPROPLIST", propList) :: Nil) => + extractProps(propList, "TOK_TABLEPROPERTY") + case _ => parseFailed("Invalid CREATE DATABASE command", node) + }.toMap + CreateDatabase(databaseName, ifNotExists.isDefined, location, comment, props)(node.source) + + // CREATE [TEMPORARY] FUNCTION [db_name.]function_name AS class_name + // [USING JAR|FILE|ARCHIVE 'file_uri' [, JAR|FILE|ARCHIVE 'file_uri'] ]; + case Token("TOK_CREATEFUNCTION", args) => + // Example format: + // + // TOK_CREATEFUNCTION + // :- db_name + // :- func_name + // :- alias + // +- TOK_RESOURCE_LIST + // :- TOK_RESOURCE_URI + // : :- TOK_JAR + // : +- '/path/to/jar' + // +- TOK_RESOURCE_URI + // :- TOK_FILE + // +- 'path/to/file' + val (funcNameArgs, otherArgs) = args.partition { + case Token("TOK_RESOURCE_LIST", _) => false + case Token("TOK_TEMPORARY", _) => false + case Token(_, Nil) => true + case _ => parseFailed("Invalid CREATE FUNCTION command", node) + } + // If database name is specified, there are 3 tokens, otherwise 2. + val (funcName, alias) = funcNameArgs match { + case Token(dbName, Nil) :: Token(fname, Nil) :: Token(aname, Nil) :: Nil => + (unquoteString(dbName) + "." + unquoteString(fname), unquoteString(aname)) + case Token(fname, Nil) :: Token(aname, Nil) :: Nil => + (unquoteString(fname), unquoteString(aname)) + case _ => + parseFailed("Invalid CREATE FUNCTION command", node) + } + // Extract other keywords, if they exist + val Seq(rList, temp) = getClauses(Seq("TOK_RESOURCE_LIST", "TOK_TEMPORARY"), otherArgs) + val resourcesMap = rList.toSeq.flatMap { + case Token("TOK_RESOURCE_LIST", resources) => + resources.map { + case Token("TOK_RESOURCE_URI", rType :: Token(rPath, Nil) :: Nil) => + val resourceType = rType match { + case Token("TOK_JAR", Nil) => "jar" + case Token("TOK_FILE", Nil) => "file" + case Token("TOK_ARCHIVE", Nil) => "archive" + case Token(f, _) => parseFailed(s"Unexpected resource format '$f'", node) + } + (resourceType, unquoteString(rPath)) + case _ => parseFailed("Invalid CREATE FUNCTION command", node) + } + case _ => parseFailed("Invalid CREATE FUNCTION command", node) + }.toMap + CreateFunction(funcName, alias, resourcesMap, temp.isDefined)(node.source) + + case Token("TOK_ALTERTABLE", alterTableArgs) => + AlterTableCommandParser.parse(node) + case Token("TOK_CREATETABLEUSING", createTableArgs) => val Seq( temp, - allowExisting, + ifNotExists, Some(tabName), tableCols, Some(Token("TOK_TABLEPROVIDER", providerNameParts)), @@ -79,30 +174,22 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly "TOK_TABLEPROVIDER", "TOK_TABLEOPTIONS", "TOK_QUERY"), createTableArgs) - val tableIdent: TableIdentifier = extractTableIdent(tabName) - val columns = tableCols.map { case Token("TOK_TABCOLLIST", fields) => StructType(fields.map(nodeToStructField)) + case _ => parseFailed("Invalid CREATE TABLE command", node) } - val provider = providerNameParts.map { case Token(name, Nil) => name + case _ => parseFailed("Invalid CREATE TABLE command", node) }.mkString(".") - - val options: Map[String, String] = tableOpts.toSeq.flatMap { - case Token("TOK_TABLEOPTIONS", options) => - options.map { - case Token("TOK_TABLEOPTION", keysAndValue) => - val key = keysAndValue.init.map(_.text).mkString(".") - val value = unquoteString(keysAndValue.last.text) - (key, value) - } + val options = tableOpts.toSeq.flatMap { + case Token("TOK_TABLEOPTIONS", opts) => extractProps(opts, "TOK_TABLEOPTION") + case _ => parseFailed("Invalid CREATE TABLE command", node) }.toMap + val asClause = tableAs.map(nodeToPlan) - val asClause = tableAs.map(nodeToPlan(_)) - - if (temp.isDefined && allowExisting.isDefined) { + if (temp.isDefined && ifNotExists.isDefined) { throw new AnalysisException( "a CREATE TEMPORARY TABLE statement does not allow IF NOT EXISTS clause.") } @@ -113,7 +200,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly "a CREATE TABLE AS SELECT statement does not allow column definitions.") } - val mode = if (allowExisting.isDefined) { + val mode = if (ifNotExists.isDefined) { SaveMode.Ignore } else if (temp.isDefined) { SaveMode.Overwrite @@ -136,7 +223,7 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly provider, temp.isDefined, options, - allowExisting.isDefined, + ifNotExists.isDefined, managedIfNoPath = false) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala new file mode 100644 index 0000000000000..58639275c111b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala @@ -0,0 +1,428 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.ExternalCatalog.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending, SortDirection} +import org.apache.spark.sql.catalyst.parser._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.types.StructType + + +/** + * Helper object to parse alter table commands. + */ +object AlterTableCommandParser { + import ParserUtils._ + + /** + * Parse the given node assuming it is an alter table command. + */ + def parse(node: ASTNode): LogicalPlan = { + node.children match { + case (tabName @ Token("TOK_TABNAME", _)) :: otherNodes => + val tableIdent = extractTableIdent(tabName) + val partSpec = getClauseOption("TOK_PARTSPEC", node.children).map(parsePartitionSpec) + matchAlterTableCommands(node, otherNodes, tableIdent, partSpec) + case _ => + parseFailed("Could not parse ALTER TABLE command", node) + } + } + + private def cleanAndUnquoteString(s: String): String = { + cleanIdentifier(unquoteString(s)) + } + + /** + * Extract partition spec from the given [[ASTNode]] as a map, assuming it exists. + * + * Expected format: + * +- TOK_PARTSPEC + * :- TOK_PARTVAL + * : :- dt + * : +- '2008-08-08' + * +- TOK_PARTVAL + * :- country + * +- 'us' + */ + private def parsePartitionSpec(node: ASTNode): Map[String, String] = { + node match { + case Token("TOK_PARTSPEC", partitions) => + partitions.map { + // Note: sometimes there's a "=", "<" or ">" between the key and the value + case Token("TOK_PARTVAL", ident :: conj :: constant :: Nil) => + (cleanAndUnquoteString(ident.text), cleanAndUnquoteString(constant.text)) + case Token("TOK_PARTVAL", ident :: constant :: Nil) => + (cleanAndUnquoteString(ident.text), cleanAndUnquoteString(constant.text)) + case Token("TOK_PARTVAL", ident :: Nil) => + (cleanAndUnquoteString(ident.text), null) + case _ => + parseFailed("Invalid ALTER TABLE command", node) + }.toMap + case _ => + parseFailed("Expected partition spec in ALTER TABLE command", node) + } + } + + /** + * Extract table properties from the given [[ASTNode]] as a map, assuming it exists. + * + * Expected format: + * +- TOK_TABLEPROPERTIES + * +- TOK_TABLEPROPLIST + * :- TOK_TABLEPROPERTY + * : :- 'test' + * : +- 'value' + * +- TOK_TABLEPROPERTY + * :- 'comment' + * +- 'new_comment' + */ + private def extractTableProps(node: ASTNode): Map[String, String] = { + node match { + case Token("TOK_TABLEPROPERTIES", propsList) => + propsList.flatMap { + case Token("TOK_TABLEPROPLIST", props) => + props.map { case Token("TOK_TABLEPROPERTY", key :: value :: Nil) => + val k = cleanAndUnquoteString(key.text) + val v = value match { + case Token("TOK_NULL", Nil) => null + case _ => cleanAndUnquoteString(value.text) + } + (k, v) + } + case _ => + parseFailed("Invalid ALTER TABLE command", node) + }.toMap + case _ => + parseFailed("Expected table properties in ALTER TABLE command", node) + } + } + + /** + * Parse an alter table command from a [[ASTNode]] into a [[LogicalPlan]]. + * This follows https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL. + * + * @param node the original [[ASTNode]] to parse. + * @param otherNodes the other [[ASTNode]]s after the first one containing the table name. + * @param tableIdent identifier of the table, parsed from the first [[ASTNode]]. + * @param partition spec identifying the partition this command is concerned with, if any. + */ + // TODO: This method is massive. Break it down. + private def matchAlterTableCommands( + node: ASTNode, + otherNodes: Seq[ASTNode], + tableIdent: TableIdentifier, + partition: Option[TablePartitionSpec]): LogicalPlan = { + otherNodes match { + // ALTER TABLE table_name RENAME TO new_table_name; + case Token("TOK_ALTERTABLE_RENAME", renameArgs) :: _ => + val tableNameClause = getClause("TOK_TABNAME", renameArgs) + val newTableIdent = extractTableIdent(tableNameClause) + AlterTableRename(tableIdent, newTableIdent)(node.source) + + // ALTER TABLE table_name SET TBLPROPERTIES ('comment' = new_comment); + case Token("TOK_ALTERTABLE_PROPERTIES", args) :: _ => + val properties = extractTableProps(args.head) + AlterTableSetProperties(tableIdent, properties)(node.source) + + // ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'key'); + case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: _ => + val properties = extractTableProps(args.head) + val ifExists = getClauseOption("TOK_IFEXISTS", args).isDefined + AlterTableUnsetProperties(tableIdent, properties, ifExists)(node.source) + + // ALTER TABLE table_name [PARTITION spec] SET SERDE serde_name [WITH SERDEPROPERTIES props]; + case Token("TOK_ALTERTABLE_SERIALIZER", Token(serdeClassName, Nil) :: serdeArgs) :: _ => + AlterTableSerDeProperties( + tableIdent, + Some(cleanAndUnquoteString(serdeClassName)), + serdeArgs.headOption.map(extractTableProps), + partition)(node.source) + + // ALTER TABLE table_name [PARTITION spec] SET SERDEPROPERTIES serde_properties; + case Token("TOK_ALTERTABLE_SERDEPROPERTIES", args) :: _ => + AlterTableSerDeProperties( + tableIdent, + None, + Some(extractTableProps(args.head)), + partition)(node.source) + + // ALTER TABLE table_name CLUSTERED BY (col, ...) [SORTED BY (col, ...)] INTO n BUCKETS; + case Token("TOK_ALTERTABLE_CLUSTER_SORT", Token("TOK_ALTERTABLE_BUCKETS", b) :: Nil) :: _ => + val clusterCols: Seq[String] = b.head match { + case Token("TOK_TABCOLNAME", children) => children.map(_.text) + case _ => parseFailed("Invalid ALTER TABLE command", node) + } + // If sort columns are specified, num buckets should be the third arg. + // If sort columns are not specified, num buckets should be the second arg. + // TODO: actually use `sortDirections` once we actually store that in the metastore + val (sortCols: Seq[String], sortDirections: Seq[SortDirection], numBuckets: Int) = { + b.tail match { + case Token("TOK_TABCOLNAME", children) :: numBucketsNode :: Nil => + val (cols, directions) = children.map { + case Token("TOK_TABSORTCOLNAMEASC", Token(col, Nil) :: Nil) => (col, Ascending) + case Token("TOK_TABSORTCOLNAMEDESC", Token(col, Nil) :: Nil) => (col, Descending) + }.unzip + (cols, directions, numBucketsNode.text.toInt) + case numBucketsNode :: Nil => + (Nil, Nil, numBucketsNode.text.toInt) + case _ => + parseFailed("Invalid ALTER TABLE command", node) + } + } + AlterTableStorageProperties( + tableIdent, + BucketSpec(numBuckets, clusterCols, sortCols))(node.source) + + // ALTER TABLE table_name NOT CLUSTERED + case Token("TOK_ALTERTABLE_CLUSTER_SORT", Token("TOK_NOT_CLUSTERED", Nil) :: Nil) :: _ => + AlterTableNotClustered(tableIdent)(node.source) + + // ALTER TABLE table_name NOT SORTED + case Token("TOK_ALTERTABLE_CLUSTER_SORT", Token("TOK_NOT_SORTED", Nil) :: Nil) :: _ => + AlterTableNotSorted(tableIdent)(node.source) + + // ALTER TABLE table_name SKEWED BY (col1, col2) + // ON ((col1_value, col2_value) [, (col1_value, col2_value), ...]) + // [STORED AS DIRECTORIES]; + case Token("TOK_ALTERTABLE_SKEWED", + Token("TOK_TABLESKEWED", + Token("TOK_TABCOLNAME", colNames) :: colValues :: rest) :: Nil) :: _ => + // Example format: + // + // +- TOK_ALTERTABLE_SKEWED + // :- TOK_TABLESKEWED + // : :- TOK_TABCOLNAME + // : : :- dt + // : : +- country + // :- TOK_TABCOLVALUE_PAIR + // : :- TOK_TABCOLVALUES + // : : :- TOK_TABCOLVALUE + // : : : :- '2008-08-08' + // : : : +- 'us' + // : :- TOK_TABCOLVALUES + // : : :- TOK_TABCOLVALUE + // : : : :- '2009-09-09' + // : : : +- 'uk' + // +- TOK_STOREASDIR + val names = colNames.map { n => cleanAndUnquoteString(n.text) } + val values = colValues match { + case Token("TOK_TABCOLVALUE", vals) => + Seq(vals.map { n => cleanAndUnquoteString(n.text) }) + case Token("TOK_TABCOLVALUE_PAIR", pairs) => + pairs.map { + case Token("TOK_TABCOLVALUES", Token("TOK_TABCOLVALUE", vals) :: Nil) => + vals.map { n => cleanAndUnquoteString(n.text) } + case _ => + parseFailed("Invalid ALTER TABLE command", node) + } + case _ => + parseFailed("Invalid ALTER TABLE command", node) + } + val storedAsDirs = rest match { + case Token("TOK_STOREDASDIRS", Nil) :: Nil => true + case _ => false + } + AlterTableSkewed( + tableIdent, + names, + values, + storedAsDirs)(node.source) + + // ALTER TABLE table_name NOT SKEWED + case Token("TOK_ALTERTABLE_SKEWED", Nil) :: _ => + AlterTableNotSkewed(tableIdent)(node.source) + + // ALTER TABLE table_name NOT STORED AS DIRECTORIES + case Token("TOK_ALTERTABLE_SKEWED", Token("TOK_STOREDASDIRS", Nil) :: Nil) :: _ => + AlterTableNotStoredAsDirs(tableIdent)(node.source) + + // ALTER TABLE table_name SET SKEWED LOCATION (col1="loc1" [, (col2, col3)="loc2", ...] ); + case Token("TOK_ALTERTABLE_SKEWED_LOCATION", + Token("TOK_SKEWED_LOCATIONS", + Token("TOK_SKEWED_LOCATION_LIST", locationMaps) :: Nil) :: Nil) :: _ => + // Expected format: + // + // +- TOK_ALTERTABLE_SKEWED_LOCATION + // +- TOK_SKEWED_LOCATIONS + // +- TOK_SKEWED_LOCATION_LIST + // :- TOK_SKEWED_LOCATION_MAP + // : :- 'col1' + // : +- 'loc1' + // +- TOK_SKEWED_LOCATION_MAP + // :- TOK_TABCOLVALUES + // : +- TOK_TABCOLVALUE + // : :- 'col2' + // : +- 'col3' + // +- 'loc2' + val skewedMaps = locationMaps.flatMap { + case Token("TOK_SKEWED_LOCATION_MAP", col :: loc :: Nil) => + col match { + case Token(const, Nil) => + Seq((cleanAndUnquoteString(const), cleanAndUnquoteString(loc.text))) + case Token("TOK_TABCOLVALUES", Token("TOK_TABCOLVALUE", keys) :: Nil) => + keys.map { k => (cleanAndUnquoteString(k.text), cleanAndUnquoteString(loc.text)) } + } + case _ => + parseFailed("Invalid ALTER TABLE command", node) + }.toMap + AlterTableSkewedLocation(tableIdent, skewedMaps)(node.source) + + // ALTER TABLE table_name ADD [IF NOT EXISTS] PARTITION spec [LOCATION 'loc1'] + // spec [LOCATION 'loc2'] ...; + case Token("TOK_ALTERTABLE_ADDPARTS", args) :: _ => + val (ifNotExists, parts) = args.head match { + case Token("TOK_IFNOTEXISTS", Nil) => (true, args.tail) + case _ => (false, args) + } + // List of (spec, location) to describe partitions to add + // Each partition spec may or may not be followed by a location + val parsedParts = new ArrayBuffer[(TablePartitionSpec, Option[String])] + parts.foreach { + case t @ Token("TOK_PARTSPEC", _) => + parsedParts += ((parsePartitionSpec(t), None)) + case Token("TOK_PARTITIONLOCATION", loc :: Nil) => + // Update the location of the last partition we just added + if (parsedParts.nonEmpty) { + val (spec, _) = parsedParts.remove(parsedParts.length - 1) + parsedParts += ((spec, Some(unquoteString(loc.text)))) + } + case _ => + parseFailed("Invalid ALTER TABLE command", node) + } + AlterTableAddPartition(tableIdent, parsedParts, ifNotExists)(node.source) + + // ALTER TABLE table_name PARTITION spec1 RENAME TO PARTITION spec2; + case Token("TOK_ALTERTABLE_RENAMEPART", spec :: Nil) :: _ => + val newPartition = parsePartitionSpec(spec) + val oldPartition = partition.getOrElse { + parseFailed("Expected old partition spec in ALTER TABLE rename partition command", node) + } + AlterTableRenamePartition(tableIdent, oldPartition, newPartition)(node.source) + + // ALTER TABLE table_name_1 EXCHANGE PARTITION spec WITH TABLE table_name_2; + case Token("TOK_ALTERTABLE_EXCHANGEPARTITION", spec :: newTable :: Nil) :: _ => + val parsedSpec = parsePartitionSpec(spec) + val newTableIdent = extractTableIdent(newTable) + AlterTableExchangePartition(tableIdent, newTableIdent, parsedSpec)(node.source) + + // ALTER TABLE table_name DROP [IF EXISTS] PARTITION spec1[, PARTITION spec2, ...] [PURGE]; + case Token("TOK_ALTERTABLE_DROPPARTS", args) :: _ => + val parts = args.collect { case p @ Token("TOK_PARTSPEC", _) => parsePartitionSpec(p) } + val ifExists = getClauseOption("TOK_IFEXISTS", args).isDefined + val purge = getClauseOption("PURGE", args).isDefined + AlterTableDropPartition(tableIdent, parts, ifExists, purge)(node.source) + + // ALTER TABLE table_name ARCHIVE PARTITION spec; + case Token("TOK_ALTERTABLE_ARCHIVE", spec :: Nil) :: _ => + AlterTableArchivePartition(tableIdent, parsePartitionSpec(spec))(node.source) + + // ALTER TABLE table_name UNARCHIVE PARTITION spec; + case Token("TOK_ALTERTABLE_UNARCHIVE", spec :: Nil) :: _ => + AlterTableUnarchivePartition(tableIdent, parsePartitionSpec(spec))(node.source) + + // ALTER TABLE table_name [PARTITION spec] SET FILEFORMAT file_format; + case Token("TOK_ALTERTABLE_FILEFORMAT", args) :: _ => + val Seq(fileFormat, genericFormat) = + getClauses(Seq("TOK_TABLEFILEFORMAT", "TOK_FILEFORMAT_GENERIC"), args) + // Note: the AST doesn't contain information about which file format is being set here. + // E.g. we can't differentiate between INPUTFORMAT and OUTPUTFORMAT if either is set. + // Right now this just stores the values, but we should figure out how to get the keys. + val fFormat = fileFormat + .map { _.children.map { n => cleanAndUnquoteString(n.text) }} + .getOrElse(Seq()) + val gFormat = genericFormat.map { f => cleanAndUnquoteString(f.children(0).text) } + AlterTableSetFileFormat(tableIdent, partition, fFormat, gFormat)(node.source) + + // ALTER TABLE table_name [PARTITION spec] SET LOCATION "loc"; + case Token("TOK_ALTERTABLE_LOCATION", Token(loc, Nil) :: Nil) :: _ => + AlterTableSetLocation(tableIdent, partition, cleanAndUnquoteString(loc))(node.source) + + // ALTER TABLE table_name TOUCH [PARTITION spec]; + case Token("TOK_ALTERTABLE_TOUCH", args) :: _ => + // Note: the partition spec, if it exists, comes after TOUCH, so `partition` should + // always be None here. Instead, we need to parse it from the TOUCH node's children. + val part = getClauseOption("TOK_PARTSPEC", args).map(parsePartitionSpec) + AlterTableTouch(tableIdent, part)(node.source) + + // ALTER TABLE table_name [PARTITION spec] COMPACT 'compaction_type'; + case Token("TOK_ALTERTABLE_COMPACT", Token(compactType, Nil) :: Nil) :: _ => + AlterTableCompact(tableIdent, partition, cleanAndUnquoteString(compactType))(node.source) + + // ALTER TABLE table_name [PARTITION spec] CONCATENATE; + case Token("TOK_ALTERTABLE_MERGEFILES", _) :: _ => + AlterTableMerge(tableIdent, partition)(node.source) + + // ALTER TABLE table_name [PARTITION spec] CHANGE [COLUMN] col_old_name col_new_name + // column_type [COMMENT col_comment] [FIRST|AFTER column_name] [CASCADE|RESTRICT]; + case Token("TOK_ALTERTABLE_RENAMECOL", oldName :: newName :: dataType :: args) :: _ => + val afterColName: Option[String] = + getClauseOption("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", args).map { ap => + ap.children match { + case Token(col, Nil) :: Nil => col + case _ => parseFailed("Invalid ALTER TABLE command", node) + } + } + val restrict = getClauseOption("TOK_RESTRICT", args).isDefined + val cascade = getClauseOption("TOK_CASCADE", args).isDefined + val comment = args.headOption.map { + case Token("TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION", _) => null + case Token("TOK_RESTRICT", _) => null + case Token("TOK_CASCADE", _) => null + case Token(commentStr, Nil) => cleanAndUnquoteString(commentStr) + case _ => parseFailed("Invalid ALTER TABLE command", node) + } + AlterTableChangeCol( + tableIdent, + partition, + oldName.text, + newName.text, + nodeToDataType(dataType), + comment, + afterColName, + restrict, + cascade)(node.source) + + // ALTER TABLE table_name [PARTITION spec] ADD COLUMNS (name type [COMMENT comment], ...) + // [CASCADE|RESTRICT] + case Token("TOK_ALTERTABLE_ADDCOLS", args) :: _ => + val columnNodes = getClause("TOK_TABCOLLIST", args).children + val columns = StructType(columnNodes.map(nodeToStructField)) + val restrict = getClauseOption("TOK_RESTRICT", args).isDefined + val cascade = getClauseOption("TOK_CASCADE", args).isDefined + AlterTableAddCol(tableIdent, partition, columns, restrict, cascade)(node.source) + + // ALTER TABLE table_name [PARTITION spec] REPLACE COLUMNS (name type [COMMENT comment], ...) + // [CASCADE|RESTRICT] + case Token("TOK_ALTERTABLE_REPLACECOLS", args) :: _ => + val columnNodes = getClause("TOK_TABCOLLIST", args).children + val columns = StructType(columnNodes.map(nodeToStructField)) + val restrict = getClauseOption("TOK_RESTRICT", args).isDefined + val cascade = getClauseOption("TOK_CASCADE", args).isDefined + AlterTableReplaceCol(tableIdent, partition, columns, restrict, cascade)(node.source) + + case _ => + parseFailed("Unsupported ALTER TABLE command", node) + } + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala new file mode 100644 index 0000000000000..9df58d214a504 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.Logging +import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.ExternalCatalog.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.execution.datasources.BucketSpec +import org.apache.spark.sql.types._ + + +// Note: The definition of these commands are based on the ones described in +// https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL + +/** + * A DDL command expected to be parsed and run in an underlying system instead of in Spark. + */ +abstract class NativeDDLCommand(val sql: String) extends RunnableCommand { + + override def run(sqlContext: SQLContext): Seq[Row] = { + sqlContext.runNativeSql(sql) + } + + override val output: Seq[Attribute] = { + Seq(AttributeReference("result", StringType, nullable = false)()) + } + +} + +case class CreateDatabase( + databaseName: String, + ifNotExists: Boolean, + path: Option[String], + comment: Option[String], + props: Map[String, String])(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class CreateFunction( + functionName: String, + alias: String, + resourcesMap: Map[String, String], + isTemp: Boolean)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableRename( + oldName: TableIdentifier, + newName: TableIdentifier)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableSetProperties( + tableName: TableIdentifier, + properties: Map[String, String])(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableUnsetProperties( + tableName: TableIdentifier, + properties: Map[String, String], + ifExists: Boolean)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableSerDeProperties( + tableName: TableIdentifier, + serdeClassName: Option[String], + serdeProperties: Option[Map[String, String]], + partition: Option[Map[String, String]])(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableStorageProperties( + tableName: TableIdentifier, + buckets: BucketSpec)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableNotClustered( + tableName: TableIdentifier)(sql: String) extends NativeDDLCommand(sql) with Logging + +case class AlterTableNotSorted( + tableName: TableIdentifier)(sql: String) extends NativeDDLCommand(sql) with Logging + +case class AlterTableSkewed( + tableName: TableIdentifier, + // e.g. (dt, country) + skewedCols: Seq[String], + // e.g. ('2008-08-08', 'us), ('2009-09-09', 'uk') + skewedValues: Seq[Seq[String]], + storedAsDirs: Boolean)(sql: String) + extends NativeDDLCommand(sql) with Logging { + + require(skewedValues.forall(_.size == skewedCols.size), + "number of columns in skewed values do not match number of skewed columns provided") +} + +case class AlterTableNotSkewed( + tableName: TableIdentifier)(sql: String) extends NativeDDLCommand(sql) with Logging + +case class AlterTableNotStoredAsDirs( + tableName: TableIdentifier)(sql: String) extends NativeDDLCommand(sql) with Logging + +case class AlterTableSkewedLocation( + tableName: TableIdentifier, + skewedMap: Map[String, String])(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableAddPartition( + tableName: TableIdentifier, + partitionSpecsAndLocs: Seq[(TablePartitionSpec, Option[String])], + ifNotExists: Boolean)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableRenamePartition( + tableName: TableIdentifier, + oldPartition: TablePartitionSpec, + newPartition: TablePartitionSpec)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableExchangePartition( + fromTableName: TableIdentifier, + toTableName: TableIdentifier, + spec: TablePartitionSpec)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableDropPartition( + tableName: TableIdentifier, + specs: Seq[TablePartitionSpec], + ifExists: Boolean, + purge: Boolean)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableArchivePartition( + tableName: TableIdentifier, + spec: TablePartitionSpec)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableUnarchivePartition( + tableName: TableIdentifier, + spec: TablePartitionSpec)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableSetFileFormat( + tableName: TableIdentifier, + partitionSpec: Option[TablePartitionSpec], + fileFormat: Seq[String], + genericFormat: Option[String])(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableSetLocation( + tableName: TableIdentifier, + partitionSpec: Option[TablePartitionSpec], + location: String)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableTouch( + tableName: TableIdentifier, + partitionSpec: Option[TablePartitionSpec])(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableCompact( + tableName: TableIdentifier, + partitionSpec: Option[TablePartitionSpec], + compactType: String)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableMerge( + tableName: TableIdentifier, + partitionSpec: Option[TablePartitionSpec])(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableChangeCol( + tableName: TableIdentifier, + partitionSpec: Option[TablePartitionSpec], + oldColName: String, + newColName: String, + dataType: DataType, + comment: Option[String], + afterColName: Option[String], + restrict: Boolean, + cascade: Boolean)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableAddCol( + tableName: TableIdentifier, + partitionSpec: Option[TablePartitionSpec], + columns: StructType, + restrict: Boolean, + cascade: Boolean)(sql: String) + extends NativeDDLCommand(sql) with Logging + +case class AlterTableReplaceCol( + tableName: TableIdentifier, + partitionSpec: Option[TablePartitionSpec], + columns: StructType, + restrict: Boolean, + cascade: Boolean)(sql: String) + extends NativeDDLCommand(sql) with Logging diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala new file mode 100644 index 0000000000000..0d632a8a130ed --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -0,0 +1,544 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.execution.SparkQl +import org.apache.spark.sql.execution.datasources.BucketSpec +import org.apache.spark.sql.types._ + +class DDLCommandSuite extends PlanTest { + private val parser = new SparkQl + + test("create database") { + val sql = + """ + |CREATE DATABASE IF NOT EXISTS database_name + |COMMENT 'database_comment' LOCATION '/home/user/db' + |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') + """.stripMargin + val parsed = parser.parsePlan(sql) + val expected = CreateDatabase( + "database_name", + ifNotExists = true, + Some("/home/user/db"), + Some("database_comment"), + Map("a" -> "a", "b" -> "b", "c" -> "c"))(sql) + comparePlans(parsed, expected) + } + + test("create function") { + val sql1 = + """ + |CREATE TEMPORARY FUNCTION helloworld as + |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar', + |FILE 'path/to/file' + """.stripMargin + val sql2 = + """ + |CREATE FUNCTION hello.world as + |'com.matthewrathbone.example.SimpleUDFExample' USING ARCHIVE '/path/to/archive', + |FILE 'path/to/file' + """.stripMargin + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val expected1 = CreateFunction( + "helloworld", + "com.matthewrathbone.example.SimpleUDFExample", + Map("jar" -> "/path/to/jar", "file" -> "path/to/file"), + isTemp = true)(sql1) + val expected2 = CreateFunction( + "hello.world", + "com.matthewrathbone.example.SimpleUDFExample", + Map("archive" -> "/path/to/archive", "file" -> "path/to/file"), + isTemp = false)(sql2) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: rename table") { + val sql = "ALTER TABLE table_name RENAME TO new_table_name" + val parsed = parser.parsePlan(sql) + val expected = AlterTableRename( + TableIdentifier("table_name", None), + TableIdentifier("new_table_name", None))(sql) + comparePlans(parsed, expected) + } + + test("alter table: alter table properties") { + val sql1 = "ALTER TABLE table_name SET TBLPROPERTIES ('test' = 'test', " + + "'comment' = 'new_comment')" + val sql2 = "ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 'test')" + val sql3 = "ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')" + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableSetProperties( + tableIdent, Map("test" -> "test", "comment" -> "new_comment"))(sql1) + val expected2 = AlterTableUnsetProperties( + tableIdent, Map("comment" -> null, "test" -> null), ifExists = false)(sql2) + val expected3 = AlterTableUnsetProperties( + tableIdent, Map("comment" -> null, "test" -> null), ifExists = true)(sql3) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + } + + test("alter table: SerDe properties") { + val sql1 = "ALTER TABLE table_name SET SERDE 'org.apache.class'" + val sql2 = + """ + |ALTER TABLE table_name SET SERDE 'org.apache.class' + |WITH SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',') + """.stripMargin + val sql3 = + """ + |ALTER TABLE table_name SET SERDEPROPERTIES ('columns'='foo,bar', + |'field.delim' = ',') + """.stripMargin + val sql4 = + """ + |ALTER TABLE table_name PARTITION (test, dt='2008-08-08', + |country='us') SET SERDE 'org.apache.class' WITH SERDEPROPERTIES ('columns'='foo,bar', + |'field.delim' = ',') + """.stripMargin + val sql5 = + """ + |ALTER TABLE table_name PARTITION (test, dt='2008-08-08', + |country='us') SET SERDEPROPERTIES ('columns'='foo,bar', 'field.delim' = ',') + """.stripMargin + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + val parsed4 = parser.parsePlan(sql4) + val parsed5 = parser.parsePlan(sql5) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableSerDeProperties( + tableIdent, Some("org.apache.class"), None, None)(sql1) + val expected2 = AlterTableSerDeProperties( + tableIdent, + Some("org.apache.class"), + Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), + None)(sql2) + val expected3 = AlterTableSerDeProperties( + tableIdent, None, Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), None)(sql3) + val expected4 = AlterTableSerDeProperties( + tableIdent, + Some("org.apache.class"), + Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), + Some(Map("test" -> null, "dt" -> "2008-08-08", "country" -> "us")))(sql4) + val expected5 = AlterTableSerDeProperties( + tableIdent, + None, + Some(Map("columns" -> "foo,bar", "field.delim" -> ",")), + Some(Map("test" -> null, "dt" -> "2008-08-08", "country" -> "us")))(sql5) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + comparePlans(parsed5, expected5) + } + + test("alter table: storage properties") { + val sql1 = "ALTER TABLE table_name CLUSTERED BY (dt, country) INTO 10 BUCKETS" + val sql2 = "ALTER TABLE table_name CLUSTERED BY (dt, country) SORTED BY " + + "(dt, country DESC) INTO 10 BUCKETS" + val sql3 = "ALTER TABLE table_name NOT CLUSTERED" + val sql4 = "ALTER TABLE table_name NOT SORTED" + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + val parsed4 = parser.parsePlan(sql4) + val tableIdent = TableIdentifier("table_name", None) + val cols = List("dt", "country") + // TODO: also test the sort directions once we keep track of that + val expected1 = AlterTableStorageProperties( + tableIdent, BucketSpec(10, cols, Nil))(sql1) + val expected2 = AlterTableStorageProperties( + tableIdent, BucketSpec(10, cols, cols))(sql2) + val expected3 = AlterTableNotClustered(tableIdent)(sql3) + val expected4 = AlterTableNotSorted(tableIdent)(sql4) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + } + + test("alter table: skewed") { + val sql1 = + """ + |ALTER TABLE table_name SKEWED BY (dt, country) ON + |(('2008-08-08', 'us'), ('2009-09-09', 'uk'), ('2010-10-10', 'cn')) STORED AS DIRECTORIES + """.stripMargin + val sql2 = + """ + |ALTER TABLE table_name SKEWED BY (dt, country) ON + |('2008-08-08', 'us') STORED AS DIRECTORIES + """.stripMargin + val sql3 = + """ + |ALTER TABLE table_name SKEWED BY (dt, country) ON + |(('2008-08-08', 'us'), ('2009-09-09', 'uk')) + """.stripMargin + val sql4 = "ALTER TABLE table_name NOT SKEWED" + val sql5 = "ALTER TABLE table_name NOT STORED AS DIRECTORIES" + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + val parsed4 = parser.parsePlan(sql4) + val parsed5 = parser.parsePlan(sql5) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableSkewed( + tableIdent, + Seq("dt", "country"), + Seq(List("2008-08-08", "us"), List("2009-09-09", "uk"), List("2010-10-10", "cn")), + storedAsDirs = true)(sql1) + val expected2 = AlterTableSkewed( + tableIdent, + Seq("dt", "country"), + Seq(List("2008-08-08", "us")), + storedAsDirs = true)(sql2) + val expected3 = AlterTableSkewed( + tableIdent, + Seq("dt", "country"), + Seq(List("2008-08-08", "us"), List("2009-09-09", "uk")), + storedAsDirs = false)(sql3) + val expected4 = AlterTableNotSkewed(tableIdent)(sql4) + val expected5 = AlterTableNotStoredAsDirs(tableIdent)(sql5) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) + comparePlans(parsed5, expected5) + } + + test("alter table: skewed location") { + val sql1 = + """ + |ALTER TABLE table_name SET SKEWED LOCATION + |('123'='location1', 'test'='location2') + """.stripMargin + val sql2 = + """ + |ALTER TABLE table_name SET SKEWED LOCATION + |(('2008-08-08', 'us')='location1', 'test'='location2') + """.stripMargin + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableSkewedLocation( + tableIdent, + Map("123" -> "location1", "test" -> "location2"))(sql1) + val expected2 = AlterTableSkewedLocation( + tableIdent, + Map("2008-08-08" -> "location1", "us" -> "location1", "test" -> "location2"))(sql2) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: add partition") { + val sql = + """ + |ALTER TABLE table_name ADD IF NOT EXISTS PARTITION + |(dt='2008-08-08', country='us') LOCATION 'location1' PARTITION + |(dt='2009-09-09', country='uk') + """.stripMargin + val parsed = parser.parsePlan(sql) + val expected = AlterTableAddPartition( + TableIdentifier("table_name", None), + Seq( + (Map("dt" -> "2008-08-08", "country" -> "us"), Some("location1")), + (Map("dt" -> "2009-09-09", "country" -> "uk"), None)), + ifNotExists = true)(sql) + comparePlans(parsed, expected) + } + + test("alter table: rename partition") { + val sql = + """ + |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') + |RENAME TO PARTITION (dt='2008-09-09', country='uk') + """.stripMargin + val parsed = parser.parsePlan(sql) + val expected = AlterTableRenamePartition( + TableIdentifier("table_name", None), + Map("dt" -> "2008-08-08", "country" -> "us"), + Map("dt" -> "2008-09-09", "country" -> "uk"))(sql) + comparePlans(parsed, expected) + } + + test("alter table: exchange partition") { + val sql = + """ + |ALTER TABLE table_name_1 EXCHANGE PARTITION + |(dt='2008-08-08', country='us') WITH TABLE table_name_2 + """.stripMargin + val parsed = parser.parsePlan(sql) + val expected = AlterTableExchangePartition( + TableIdentifier("table_name_1", None), + TableIdentifier("table_name_2", None), + Map("dt" -> "2008-08-08", "country" -> "us"))(sql) + comparePlans(parsed, expected) + } + + test("alter table: drop partitions") { + val sql1 = + """ + |ALTER TABLE table_name DROP IF EXISTS PARTITION + |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') + """.stripMargin + val sql2 = + """ + |ALTER TABLE table_name DROP PARTITION + |(dt='2008-08-08', country='us'), PARTITION (dt='2009-09-09', country='uk') PURGE + """.stripMargin + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableDropPartition( + tableIdent, + Seq( + Map("dt" -> "2008-08-08", "country" -> "us"), + Map("dt" -> "2009-09-09", "country" -> "uk")), + ifExists = true, + purge = false)(sql1) + val expected2 = AlterTableDropPartition( + tableIdent, + Seq( + Map("dt" -> "2008-08-08", "country" -> "us"), + Map("dt" -> "2009-09-09", "country" -> "uk")), + ifExists = false, + purge = true)(sql2) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: archive partition") { + val sql = "ALTER TABLE table_name ARCHIVE PARTITION (dt='2008-08-08', country='us')" + val parsed = parser.parsePlan(sql) + val expected = AlterTableArchivePartition( + TableIdentifier("table_name", None), + Map("dt" -> "2008-08-08", "country" -> "us"))(sql) + comparePlans(parsed, expected) + } + + test("alter table: unarchive partition") { + val sql = "ALTER TABLE table_name UNARCHIVE PARTITION (dt='2008-08-08', country='us')" + val parsed = parser.parsePlan(sql) + val expected = AlterTableUnarchivePartition( + TableIdentifier("table_name", None), + Map("dt" -> "2008-08-08", "country" -> "us"))(sql) + comparePlans(parsed, expected) + } + + test("alter table: set file format") { + val sql1 = + """ + |ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' + |OUTPUTFORMAT 'test' SERDE 'test' INPUTDRIVER 'test' OUTPUTDRIVER 'test' + """.stripMargin + val sql2 = "ALTER TABLE table_name SET FILEFORMAT INPUTFORMAT 'test' " + + "OUTPUTFORMAT 'test' SERDE 'test'" + val sql3 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + + "SET FILEFORMAT PARQUET" + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableSetFileFormat( + tableIdent, + None, + List("test", "test", "test", "test", "test"), + None)(sql1) + val expected2 = AlterTableSetFileFormat( + tableIdent, + None, + List("test", "test", "test"), + None)(sql2) + val expected3 = AlterTableSetFileFormat( + tableIdent, + Some(Map("dt" -> "2008-08-08", "country" -> "us")), + Seq(), + Some("PARQUET"))(sql3) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + } + + test("alter table: set location") { + val sql1 = "ALTER TABLE table_name SET LOCATION 'new location'" + val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + + "SET LOCATION 'new location'" + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableSetLocation( + tableIdent, + None, + "new location")(sql1) + val expected2 = AlterTableSetLocation( + tableIdent, + Some(Map("dt" -> "2008-08-08", "country" -> "us")), + "new location")(sql2) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: touch") { + val sql1 = "ALTER TABLE table_name TOUCH" + val sql2 = "ALTER TABLE table_name TOUCH PARTITION (dt='2008-08-08', country='us')" + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableTouch( + tableIdent, + None)(sql1) + val expected2 = AlterTableTouch( + tableIdent, + Some(Map("dt" -> "2008-08-08", "country" -> "us")))(sql2) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: compact") { + val sql1 = "ALTER TABLE table_name COMPACT 'compaction_type'" + val sql2 = + """ + |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') + |COMPACT 'MAJOR' + """.stripMargin + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableCompact( + tableIdent, + None, + "compaction_type")(sql1) + val expected2 = AlterTableCompact( + tableIdent, + Some(Map("dt" -> "2008-08-08", "country" -> "us")), + "MAJOR")(sql2) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: concatenate") { + val sql1 = "ALTER TABLE table_name CONCATENATE" + val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') CONCATENATE" + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableMerge(tableIdent, None)(sql1) + val expected2 = AlterTableMerge( + tableIdent, Some(Map("dt" -> "2008-08-08", "country" -> "us")))(sql2) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + + test("alter table: change column name/type/position/comment") { + val sql1 = "ALTER TABLE table_name CHANGE col_old_name col_new_name INT" + val sql2 = + """ + |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT + |COMMENT 'col_comment' FIRST CASCADE + """.stripMargin + val sql3 = + """ + |ALTER TABLE table_name CHANGE COLUMN col_old_name col_new_name INT + |COMMENT 'col_comment' AFTER column_name RESTRICT + """.stripMargin + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val parsed3 = parser.parsePlan(sql3) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableChangeCol( + tableName = tableIdent, + partitionSpec = None, + oldColName = "col_old_name", + newColName = "col_new_name", + dataType = IntegerType, + comment = None, + afterColName = None, + restrict = false, + cascade = false)(sql1) + val expected2 = AlterTableChangeCol( + tableName = tableIdent, + partitionSpec = None, + oldColName = "col_old_name", + newColName = "col_new_name", + dataType = IntegerType, + comment = Some("col_comment"), + afterColName = None, + restrict = false, + cascade = true)(sql2) + val expected3 = AlterTableChangeCol( + tableName = tableIdent, + partitionSpec = None, + oldColName = "col_old_name", + newColName = "col_new_name", + dataType = IntegerType, + comment = Some("col_comment"), + afterColName = Some("column_name"), + restrict = true, + cascade = false)(sql3) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + comparePlans(parsed3, expected3) + } + + test("alter table: add/replace columns") { + val sql1 = + """ + |ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') + |ADD COLUMNS (new_col1 INT COMMENT 'test_comment', new_col2 LONG + |COMMENT 'test_comment2') CASCADE + """.stripMargin + val sql2 = + """ + |ALTER TABLE table_name REPLACE COLUMNS (new_col1 INT + |COMMENT 'test_comment', new_col2 LONG COMMENT 'test_comment2') RESTRICT + """.stripMargin + val parsed1 = parser.parsePlan(sql1) + val parsed2 = parser.parsePlan(sql2) + val meta1 = new MetadataBuilder().putString("comment", "test_comment").build() + val meta2 = new MetadataBuilder().putString("comment", "test_comment2").build() + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableAddCol( + tableIdent, + Some(Map("dt" -> "2008-08-08", "country" -> "us")), + StructType(Seq( + StructField("new_col1", IntegerType, nullable = true, meta1), + StructField("new_col2", LongType, nullable = true, meta2))), + restrict = false, + cascade = true)(sql1) + val expected2 = AlterTableReplaceCol( + tableIdent, + None, + StructType(Seq( + StructField("new_col1", IntegerType, nullable = true, meta1), + StructField("new_col2", LongType, nullable = true, meta2))), + restrict = true, + cascade = false)(sql2) + comparePlans(parsed1, expected1) + comparePlans(parsed2, expected2) + } + +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 69669d79be2d8..081d849a88886 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -535,6 +535,15 @@ class HiveContext private[hive]( } } + /** + * Executes a SQL query without parsing it, but instead passing it directly to Hive. + * This is currently only used for DDLs and will be removed as soon as Spark can parse + * all supported Hive DDLs itself. + */ + protected[sql] override def runNativeSql(sqlText: String): Seq[Row] = { + runSqlHive(sqlText).map { s => Row(s) } + } + /** Extends QueryExecution with hive specific features. */ protected[sql] class QueryExecution(logicalPlan: LogicalPlan) extends org.apache.spark.sql.execution.QueryExecution(this, logicalPlan) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 0bdebdc5fd741..56acb87c800d3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -88,29 +88,14 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging "TOK_ALTERDATABASE_PROPERTIES", "TOK_ALTERINDEX_PROPERTIES", "TOK_ALTERINDEX_REBUILD", - "TOK_ALTERTABLE", - "TOK_ALTERTABLE_ADDCOLS", - "TOK_ALTERTABLE_ADDPARTS", "TOK_ALTERTABLE_ALTERPARTS", - "TOK_ALTERTABLE_ARCHIVE", - "TOK_ALTERTABLE_CLUSTER_SORT", - "TOK_ALTERTABLE_DROPPARTS", "TOK_ALTERTABLE_PARTITION", - "TOK_ALTERTABLE_PROPERTIES", - "TOK_ALTERTABLE_RENAME", - "TOK_ALTERTABLE_RENAMECOL", - "TOK_ALTERTABLE_REPLACECOLS", - "TOK_ALTERTABLE_SKEWED", - "TOK_ALTERTABLE_TOUCH", - "TOK_ALTERTABLE_UNARCHIVE", "TOK_ALTERVIEW_ADDPARTS", "TOK_ALTERVIEW_AS", "TOK_ALTERVIEW_DROPPARTS", "TOK_ALTERVIEW_PROPERTIES", "TOK_ALTERVIEW_RENAME", - "TOK_CREATEDATABASE", - "TOK_CREATEFUNCTION", "TOK_CREATEINDEX", "TOK_CREATEMACRO", "TOK_CREATEROLE", @@ -164,7 +149,8 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging protected val noExplainCommands = Seq( "TOK_DESCTABLE", "TOK_SHOWTABLES", - "TOK_TRUNCATETABLE" // truncate table" is a NativeCommand, does not need to explain. + "TOK_TRUNCATETABLE", // truncate table" is a NativeCommand, does not need to explain. + "TOK_ALTERTABLE" ) ++ nativeCommands /** From 2ef4c5963bff3574fe17e669d703b25ddd064e5d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 11 Mar 2016 15:39:21 -0800 Subject: [PATCH 07/21] [SPARK-13830] prefer block manager than direct result for large result ## What changes were proposed in this pull request? The current RPC can't handle large blocks very well, it's very slow to fetch 100M block (about 1 minute). Once switch to block manager to fetch that, it took about 10 seconds (still could be improved). ## How was this patch tested? existing unit tests. Author: Davies Liu Closes #11659 from davies/direct_result. --- .../main/scala/org/apache/spark/executor/Executor.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index e88d6cd08998c..07e3c12bc9bc3 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -97,9 +97,11 @@ private[spark] class Executor( // Set the classloader for serializer env.serializer.setDefaultClassLoader(replClassLoader) - // Max RPC message size. If task result is bigger than this, we use the block manager + // Max size of direct result. If task result is bigger than this, we use the block manager // to send the result back. - private val maxRpcMessageSize = RpcUtils.maxMessageSizeBytes(conf) + private val maxDirectResultSize = Math.min( + conf.getSizeAsBytes("spark.task.maxDirectResultSize", 1L << 20), + RpcUtils.maxMessageSizeBytes(conf)) // Limit of bytes for total size of results (default is 1GB) private val maxResultSize = Utils.getMaxResultSize(conf) @@ -279,6 +281,7 @@ private[spark] class Executor( // Note: accumulator updates must be collected after TaskMetrics is updated val accumUpdates = task.collectAccumulatorUpdates() + // TODO: do not serialize value twice val directResult = new DirectTaskResult(valueBytes, accumUpdates) val serializedDirectResult = ser.serialize(directResult) val resultSize = serializedDirectResult.limit @@ -290,7 +293,7 @@ private[spark] class Executor( s"(${Utils.bytesToString(resultSize)} > ${Utils.bytesToString(maxResultSize)}), " + s"dropping it.") ser.serialize(new IndirectTaskResult[Any](TaskResultBlockId(taskId), resultSize)) - } else if (resultSize >= maxRpcMessageSize) { + } else if (resultSize > maxDirectResultSize) { val blockId = TaskResultBlockId(taskId) env.blockManager.putBytes( blockId, serializedDirectResult, StorageLevel.MEMORY_AND_DISK_SER) From ba8c86d06f5968c1af4db8dd9a458005bc5f214c Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 12 Mar 2016 00:48:36 -0800 Subject: [PATCH 08/21] [SPARK-13671] [SPARK-13311] [SQL] Use different physical plans for RDD and data sources ## What changes were proposed in this pull request? This PR split the PhysicalRDD into two classes, PhysicalRDD and PhysicalScan. PhysicalRDD is used for DataFrames that is created from existing RDD. PhysicalScan is used for DataFrame that is created from data sources. This enable use to apply different optimization on both of them. Also fix the problem for sameResult() on two DataSourceScan. Also fix the equality check to toString for `In`. It's better to use Seq there, but we can't break this public API (sad). ## How was this patch tested? Existing tests. Manually tested with TPCDS query Q59 and Q64, all those duplicated exchanges can be re-used now, also saw there are 40+% performance improvement (saving half of the scan). Author: Davies Liu Closes #11514 from davies/existing_rdd. --- python/pyspark/sql/dataframe.py | 3 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 12 +- .../spark/sql/execution/ExistingRDD.scala | 107 +++++++++++------- .../sql/execution/WholeStageCodegen.scala | 1 + .../datasources/DataSourceStrategy.scala | 8 +- .../sql/execution/ui/SparkPlanGraph.scala | 6 +- .../apache/spark/sql/sources/filters.scala | 19 +++- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 6 +- .../spark/sql/sources/FilteredScanSuite.scala | 2 +- .../spark/sql/sources/PrunedScanSuite.scala | 2 +- .../apache/spark/sql/hive/parquetSuites.scala | 4 +- .../spark/sql/sources/BucketedReadSuite.scala | 4 +- 12 files changed, 110 insertions(+), 64 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 99d665fafec89..7008e8fadffc3 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -173,8 +173,7 @@ def explain(self, extended=False): >>> df.explain() == Physical Plan == - WholeStageCodegen - : +- Scan ExistingRDD[age#0,name#1] + Scan ExistingRDD[age#0,name#1] >>> df.explain(True) == Parsed Logical Plan == diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index c222571a3464b..920e989d058dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -280,12 +280,12 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT * can do better should override this function. */ def sameResult(plan: PlanType): Boolean = { - val canonicalizedLeft = this.canonicalized - val canonicalizedRight = plan.canonicalized - canonicalizedLeft.getClass == canonicalizedRight.getClass && - canonicalizedLeft.children.size == canonicalizedRight.children.size && - canonicalizedLeft.cleanArgs == canonicalizedRight.cleanArgs && - (canonicalizedLeft.children, canonicalizedRight.children).zipped.forall(_ sameResult _) + val left = this.canonicalized + val right = plan.canonicalized + left.getClass == right.getClass && + left.children.size == right.children.size && + left.cleanArgs == right.cleanArgs && + (left.children, right.children).zipped.forall(_ sameResult _) } /** 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 3662ed74d2eae..d363cb000d39a 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 @@ -101,17 +101,76 @@ private[sql] case class LogicalRDD( private[sql] case class PhysicalRDD( output: Seq[Attribute], rdd: RDD[InternalRow], - override val nodeName: String, - override val metadata: Map[String, String] = Map.empty, - isUnsafeRow: Boolean = false, - override val outputPartitioning: Partitioning = UnknownPartitioning(0)) + override val nodeName: String) extends LeafNode { + + private[sql] override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) + + protected override def doExecute(): RDD[InternalRow] = { + val numOutputRows = longMetric("numOutputRows") + rdd.mapPartitionsInternal { iter => + val proj = UnsafeProjection.create(schema) + iter.map { r => + numOutputRows += 1 + proj(r) + } + } + } + + override def simpleString: String = { + s"Scan $nodeName${output.mkString("[", ",", "]")}" + } +} + +/** Physical plan node for scanning data from a relation. */ +private[sql] case class DataSourceScan( + output: Seq[Attribute], + rdd: RDD[InternalRow], + @transient relation: BaseRelation, + override val metadata: Map[String, String] = Map.empty) extends LeafNode with CodegenSupport { + override val nodeName: String = relation.toString + + // Ignore rdd when checking results + override def sameResult(plan: SparkPlan ): Boolean = plan match { + case other: DataSourceScan => relation == other.relation && metadata == other.metadata + case _ => false + } + private[sql] override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows")) + val outputUnsafeRows = relation match { + case r: HadoopFsRelation if r.fileFormat.isInstanceOf[ParquetSource] => + !SQLContext.getActive().get.conf.getConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED) + case _: HadoopFsRelation => true + case _ => false + } + + override val outputPartitioning = { + val bucketSpec = relation match { + // TODO: this should be closer to bucket planning. + case r: HadoopFsRelation if r.sqlContext.conf.bucketingEnabled() => r.bucketSpec + case _ => None + } + + def toAttribute(colName: String): Attribute = output.find(_.name == colName).getOrElse { + throw new AnalysisException(s"bucket column $colName not found in existing columns " + + s"(${output.map(_.name).mkString(", ")})") + } + + bucketSpec.map { spec => + val numBuckets = spec.numBuckets + val bucketColumns = spec.bucketColumnNames.map(toAttribute) + HashPartitioning(bucketColumns, numBuckets) + }.getOrElse { + UnknownPartitioning(0) + } + } + protected override def doExecute(): RDD[InternalRow] = { - val unsafeRow = if (isUnsafeRow) { + val unsafeRow = if (outputUnsafeRows) { rdd } else { rdd.mapPartitionsInternal { iter => @@ -187,7 +246,7 @@ private[sql] case class PhysicalRDD( ctx.INPUT_ROW = row ctx.currentVars = null val columns2 = exprs.map(_.gen(ctx)) - val inputRow = if (isUnsafeRow) row else null + val inputRow = if (outputUnsafeRows) row else null val scanRows = ctx.freshName("processRows") ctx.addNewFunction(scanRows, s""" @@ -221,42 +280,8 @@ private[sql] case class PhysicalRDD( } } -private[sql] object PhysicalRDD { +private[sql] object DataSourceScan { // Metadata keys val INPUT_PATHS = "InputPaths" val PUSHED_FILTERS = "PushedFilters" - - def createFromDataSource( - output: Seq[Attribute], - rdd: RDD[InternalRow], - relation: BaseRelation, - metadata: Map[String, String] = Map.empty): PhysicalRDD = { - - val outputUnsafeRows = relation match { - case r: HadoopFsRelation if r.fileFormat.isInstanceOf[ParquetSource] => - !SQLContext.getActive().get.conf.getConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED) - case _: HadoopFsRelation => true - case _ => false - } - - val bucketSpec = relation match { - // TODO: this should be closer to bucket planning. - case r: HadoopFsRelation if r.sqlContext.conf.bucketingEnabled() => r.bucketSpec - case _ => None - } - - def toAttribute(colName: String): Attribute = output.find(_.name == colName).getOrElse { - throw new AnalysisException(s"bucket column $colName not found in existing columns " + - s"(${output.map(_.name).mkString(", ")})") - } - - bucketSpec.map { spec => - val numBuckets = spec.numBuckets - val bucketColumns = spec.bucketColumnNames.map(toAttribute) - val partitioning = HashPartitioning(bucketColumns, numBuckets) - PhysicalRDD(output, rdd, relation.toString, metadata, outputUnsafeRows, partitioning) - }.getOrElse { - PhysicalRDD(output, rdd, relation.toString, metadata, outputUnsafeRows) - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala index 52c2971b73f1b..8fb4705581a38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegen.scala @@ -41,6 +41,7 @@ trait CodegenSupport extends SparkPlan { case _: BroadcastHashJoin => "bhj" case _: SortMergeJoin => "smj" case _: PhysicalRDD => "rdd" + case _: DataSourceScan => "scan" case _ => nodeName.toLowerCase } 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 2944a8f86f169..1adf3b6676555 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 @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.PhysicalRDD.{INPUT_PATHS, PUSHED_FILTERS} +import org.apache.spark.sql.execution.DataSourceScan.{INPUT_PATHS, PUSHED_FILTERS} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.ExecutedCommand import org.apache.spark.sql.execution.vectorized.{ColumnarBatch, ColumnVectorUtils} @@ -239,7 +239,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { } case l @ LogicalRelation(baseRelation: TableScan, _, _) => - execution.PhysicalRDD.createFromDataSource( + execution.DataSourceScan( l.output, toCatalystRDD(l, baseRelation.buildScan()), baseRelation) :: Nil case i @ logical.InsertIntoTable(l @ LogicalRelation(t: InsertableRelation, _, _), @@ -639,7 +639,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { // Don't request columns that are only referenced by pushed filters. .filterNot(handledSet.contains) - val scan = execution.PhysicalRDD.createFromDataSource( + val scan = execution.DataSourceScan( projects.map(_.toAttribute), scanBuilder(requestedColumns, candidatePredicates, pushedFilters), relation.relation, metadata) @@ -649,7 +649,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { val requestedColumns = (projectSet ++ filterSet -- handledSet).map(relation.attributeMap).toSeq - val scan = execution.PhysicalRDD.createFromDataSource( + val scan = execution.DataSourceScan( requestedColumns, scanBuilder(requestedColumns, candidatePredicates, pushedFilters), relation.relation, metadata) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala index 94d318e702789..8a36d3224003a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SparkPlanGraph.scala @@ -93,6 +93,10 @@ private[sql] object SparkPlanGraph { case "Subquery" if subgraph != null => // Subquery should not be included in WholeStageCodegen buildSparkPlanGraphNode(planInfo, nodeIdGenerator, nodes, edges, parent, null, exchanges) + case "ReusedExchange" => + // Point to the re-used exchange + val node = exchanges(planInfo.children.head) + edges += SparkPlanGraphEdge(node.id, parent.id) case name => val metrics = planInfo.metrics.map { metric => SQLPlanMetric(metric.name, metric.accumulatorId, @@ -106,7 +110,7 @@ private[sql] object SparkPlanGraph { } else { subgraph.nodes += node } - if (name == "ShuffleExchange" || name == "BroadcastExchange") { + if (name.contains("Exchange")) { exchanges += planInfo -> node } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala index 3780cbbcc9631..9130e77ea5724 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala @@ -82,7 +82,24 @@ case class LessThanOrEqual(attribute: String, value: Any) extends Filter * * @since 1.3.0 */ -case class In(attribute: String, values: Array[Any]) extends Filter +case class In(attribute: String, values: Array[Any]) extends Filter { + override def hashCode(): Int = { + var h = attribute.hashCode + values.foreach { v => + h *= 41 + h += v.hashCode() + } + h + } + override def equals(o: Any): Boolean = o match { + case In(a, vs) => + a == attribute && vs.length == values.length && vs.zip(values).forall(x => x._1 == x._2) + case _ => false + } + override def toString: String = { + s"In($attribute, [${values.mkString(",")}]" + } +} /** * A filter that evaluates to `true` iff the attribute evaluates to null. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index dfffa582120cc..1ef517324d7cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -26,7 +26,7 @@ import org.scalatest.{BeforeAndAfter, PrivateMethodTester} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.{DataFrame, Row} -import org.apache.spark.sql.execution.PhysicalRDD +import org.apache.spark.sql.execution.DataSourceScan import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD @@ -210,8 +210,8 @@ class JDBCSuite extends SparkFunSuite // the plan only has PhysicalRDD to scan JDBCRelation. assert(parentPlan.isInstanceOf[org.apache.spark.sql.execution.WholeStageCodegen]) val node = parentPlan.asInstanceOf[org.apache.spark.sql.execution.WholeStageCodegen] - assert(node.child.isInstanceOf[org.apache.spark.sql.execution.PhysicalRDD]) - assert(node.child.asInstanceOf[PhysicalRDD].nodeName.contains("JDBCRelation")) + assert(node.child.isInstanceOf[org.apache.spark.sql.execution.DataSourceScan]) + assert(node.child.asInstanceOf[DataSourceScan].nodeName.contains("JDBCRelation")) df } assert(checkPushdown(sql("SELECT * FROM foobar WHERE THEID < 1")).collect().size == 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index 2ff79a2316bdc..19e34b45bff67 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -312,7 +312,7 @@ class FilteredScanSuite extends DataSourceTest with SharedSQLContext with Predic try { val queryExecution = sql(sqlString).queryExecution val rawPlan = queryExecution.executedPlan.collect { - case p: execution.PhysicalRDD => p + case p: execution.DataSourceScan => p } match { case Seq(p) => p case _ => fail(s"More than one PhysicalRDD found\n$queryExecution") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index db722975379a2..62f991fc5dc61 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -124,7 +124,7 @@ class PrunedScanSuite extends DataSourceTest with SharedSQLContext { try { val queryExecution = sql(sqlString).queryExecution val rawPlan = queryExecution.executedPlan.collect { - case p: execution.PhysicalRDD => p + case p: execution.DataSourceScan => p } match { case Seq(p) => p case _ => fail(s"More than one PhysicalRDD found\n$queryExecution") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index a0f09d6c4a36e..8fdbbd94c807e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hive import java.io.File import org.apache.spark.sql._ -import org.apache.spark.sql.execution.PhysicalRDD +import org.apache.spark.sql.execution.DataSourceScan import org.apache.spark.sql.execution.command.ExecutedCommand import org.apache.spark.sql.execution.datasources.{InsertIntoDataSource, InsertIntoHadoopFsRelation, LogicalRelation} import org.apache.spark.sql.hive.execution.HiveTableScan @@ -196,7 +196,7 @@ class ParquetMetastoreSuite extends ParquetPartitioningTest { }.isEmpty) assert( sql("SELECT * FROM normal_parquet").queryExecution.sparkPlan.collect { - case _: PhysicalRDD => true + case _: DataSourceScan => true }.nonEmpty) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 35573f62dc633..a0be55cfba94c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -22,7 +22,7 @@ import java.io.File import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.execution.PhysicalRDD +import org.apache.spark.sql.execution.DataSourceScan import org.apache.spark.sql.execution.datasources.{BucketSpec, DataSourceStrategy} import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.execution.joins.SortMergeJoin @@ -93,7 +93,7 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet // Filter could hide the bug in bucket pruning. Thus, skipping all the filters val plan = bucketedDataFrame.filter(filterCondition).queryExecution.executedPlan - val rdd = plan.find(_.isInstanceOf[PhysicalRDD]) + val rdd = plan.find(_.isInstanceOf[DataSourceScan]) assert(rdd.isDefined, plan) val checkedResult = rdd.get.execute().mapPartitionsWithIndex { case (index, iter) => From 4eace4d384f0e12b4934019d8654b5e3886ddaef Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 12 Mar 2016 11:24:50 -0800 Subject: [PATCH 09/21] [SPARK-13828][SQL] Bring back stack trace of AnalysisException thrown from QueryExecution.assertAnalyzed PR #11443 added an extra `plan: Option[LogicalPlan]` argument to `AnalysisException` and attached partially analyzed plan to thrown `AnalysisException` in `QueryExecution.assertAnalyzed()`. However, the original stack trace wasn't properly inherited. This PR fixes this issue by inheriting the stack trace. A test case is added to verify that the first entry of `AnalysisException` stack trace isn't from `QueryExecution`. Author: Cheng Lian Closes #11677 from liancheng/analysis-exception-stacktrace. --- .../apache/spark/sql/execution/QueryExecution.scala | 4 +++- .../scala/org/apache/spark/sql/DataFrameSuite.scala | 11 ++++++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 19ab3ea132ecc..9e60c1cd6141c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -33,7 +33,9 @@ class QueryExecution(val sqlContext: SQLContext, val logical: LogicalPlan) { def assertAnalyzed(): Unit = try sqlContext.analyzer.checkAnalysis(analyzed) catch { case e: AnalysisException => - throw new AnalysisException(e.message, e.line, e.startPosition, Some(analyzed)) + val ae = new AnalysisException(e.message, e.line, e.startPosition, Some(analyzed)) + ae.setStackTrace(e.getStackTrace) + throw ae } lazy val analyzed: LogicalPlan = sqlContext.analyzer.execute(logical) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 46cd380a797e2..e6e27ec413bb2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -25,7 +25,8 @@ import scala.util.Random import org.scalatest.Matchers._ import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.plans.logical.{BroadcastHint, OneRowRelation, Union} +import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Union} +import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.aggregate.TungstenAggregate import org.apache.spark.sql.execution.exchange.{BroadcastExchange, ReusedExchange, ShuffleExchange} import org.apache.spark.sql.functions._ @@ -1366,4 +1367,12 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { // another invalid table name test as below intercept[AnalysisException](df.registerTempTable("table!#")) } + + test("assertAnalyzed shouldn't replace original stack trace") { + val e = intercept[AnalysisException] { + sqlContext.range(1).select('id as 'a, 'id as 'b).groupBy('a).agg('b) + } + + assert(e.getStackTrace.head.getClassName != classOf[QueryExecution].getName) + } } From c079420d7c55d8972db716a2695a5ddd606d11cd Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 13 Mar 2016 12:02:52 +0800 Subject: [PATCH 10/21] [SPARK-13841][SQL] Removes Dataset.collectRows()/takeRows() ## What changes were proposed in this pull request? This PR removes two methods, `collectRows()` and `takeRows()`, from `Dataset[T]`. These methods were added in PR #11443, and were later considered not useful. ## How was this patch tested? Existing tests should do the work. Author: Cheng Lian Closes #11678 from liancheng/remove-collect-rows-and-take-rows. --- .../examples/ml/JavaBinarizerExample.java | 2 +- .../ml/JavaCrossValidatorExample.java | 2 +- .../examples/ml/JavaDeveloperApiExample.java | 2 +- .../JavaEstimatorTransformerParamExample.java | 3 +- ...delSelectionViaCrossValidationExample.java | 2 +- .../spark/examples/ml/JavaNGramExample.java | 2 +- .../examples/ml/JavaPipelineExample.java | 2 +- .../ml/JavaPolynomialExpansionExample.java | 5 ++- .../examples/ml/JavaSimpleParamsExample.java | 3 +- .../JavaSimpleTextClassificationPipeline.java | 2 +- .../spark/examples/ml/JavaTfIdfExample.java | 2 +- .../examples/ml/JavaTokenizerExample.java | 2 +- .../examples/ml/JavaWord2VecExample.java | 2 +- .../ml/feature/JavaVectorSlicerSuite.java | 2 +- .../org/apache/spark/sql/DataFrame.scala | 18 --------- .../spark/sql/JavaApplySchemaSuite.java | 4 +- .../apache/spark/sql/JavaDataFrameSuite.java | 39 ++++++++++--------- 17 files changed, 40 insertions(+), 54 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java index d554377975b1b..0a6e9c2a1f93c 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBinarizerExample.java @@ -58,7 +58,7 @@ public static void main(String[] args) { .setThreshold(0.5); Dataset binarizedDataFrame = binarizer.transform(continuousDataFrame); Dataset binarizedFeatures = binarizedDataFrame.select("binarized_feature"); - for (Row r : binarizedFeatures.collectRows()) { + for (Row r : binarizedFeatures.collectAsList()) { Double binarized_value = r.getDouble(0); System.out.println(binarized_value); } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java index 90bc94c45bbf9..07edeb3e521c3 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java @@ -117,7 +117,7 @@ public static void main(String[] args) { // Make predictions on test documents. cvModel uses the best model found (lrModel). Dataset predictions = cvModel.transform(test); - for (Row r: predictions.select("id", "text", "probability", "prediction").collectRows()) { + for (Row r: predictions.select("id", "text", "probability", "prediction").collectAsList()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + ", prediction=" + r.get(3)); } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java index e8ae100d68529..8a10dd48aa72f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java @@ -85,7 +85,7 @@ public static void main(String[] args) throws Exception { // Make predictions on test documents. cvModel uses the best model found (lrModel). Dataset results = model.transform(test); double sumPredictions = 0; - for (Row r : results.select("features", "label", "prediction").collectRows()) { + for (Row r : results.select("features", "label", "prediction").collectAsList()) { sumPredictions += r.getDouble(2); } if (sumPredictions != 0.0) { diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java index f13698ae5e07e..604b193dd489b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaEstimatorTransformerParamExample.java @@ -100,7 +100,8 @@ public static void main(String[] args) { // Note that model2.transform() outputs a 'myProbability' column instead of the usual // 'probability' column since we renamed the lr.probabilityCol parameter previously. Dataset results = model2.transform(test); - for (Row r : results.select("features", "label", "myProbability", "prediction").collectRows()) { + Dataset rows = results.select("features", "label", "myProbability", "prediction"); + for (Row r: rows.collectAsList()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) + ", prediction=" + r.get(3)); } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java index e394605db70ea..c4122d1247a94 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaModelSelectionViaCrossValidationExample.java @@ -111,7 +111,7 @@ public static void main(String[] args) { // Make predictions on test documents. cvModel uses the best model found (lrModel). Dataset predictions = cvModel.transform(test); - for (Row r : predictions.select("id", "text", "probability", "prediction").collectRows()) { + for (Row r : predictions.select("id", "text", "probability", "prediction").collectAsList()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + ", prediction=" + r.get(3)); } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java index 0305f737ca94c..608bd80285655 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaNGramExample.java @@ -60,7 +60,7 @@ public static void main(String[] args) { Dataset ngramDataFrame = ngramTransformer.transform(wordDataFrame); - for (Row r : ngramDataFrame.select("ngrams", "label").takeRows(3)) { + for (Row r : ngramDataFrame.select("ngrams", "label").takeAsList(3)) { java.util.List ngrams = r.getList(0); for (String ngram : ngrams) System.out.print(ngram + " --- "); System.out.println(); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java index 6ae418d564d1f..305420f208b79 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPipelineExample.java @@ -80,7 +80,7 @@ public static void main(String[] args) { // Make predictions on test documents. Dataset predictions = model.transform(test); - for (Row r : predictions.select("id", "text", "probability", "prediction").collectRows()) { + for (Row r : predictions.select("id", "text", "probability", "prediction").collectAsList()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + ", prediction=" + r.get(3)); } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java index 5a4064c604301..48fc3c8acb0c0 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPolynomialExpansionExample.java @@ -23,6 +23,7 @@ // $example on$ import java.util.Arrays; +import java.util.List; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.feature.PolynomialExpansion; @@ -61,8 +62,8 @@ public static void main(String[] args) { Dataset df = jsql.createDataFrame(data, schema); Dataset polyDF = polyExpansion.transform(df); - Row[] row = polyDF.select("polyFeatures").takeRows(3); - for (Row r : row) { + List rows = polyDF.select("polyFeatures").takeAsList(3); + for (Row r : rows) { System.out.println(r.get(0)); } // $example off$ diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index 52bb4ec050376..cb911ef5ef586 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -103,7 +103,8 @@ public static void main(String[] args) { // Note that model2.transform() outputs a 'myProbability' column instead of the usual // 'probability' column since we renamed the lr.probabilityCol parameter previously. Dataset results = model2.transform(test); - for (Row r: results.select("features", "label", "myProbability", "prediction").collectRows()) { + Dataset rows = results.select("features", "label", "myProbability", "prediction"); + for (Row r: rows.collectAsList()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) + ", prediction=" + r.get(3)); } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java index 9bd543c44f983..a18a60f448166 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java @@ -84,7 +84,7 @@ public static void main(String[] args) { // Make predictions on test documents. Dataset predictions = model.transform(test); - for (Row r: predictions.select("id", "text", "probability", "prediction").collectRows()) { + for (Row r: predictions.select("id", "text", "probability", "prediction").collectAsList()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> prob=" + r.get(2) + ", prediction=" + r.get(3)); } diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java index fd1ce424bf8c4..37a3d0d84dae2 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTfIdfExample.java @@ -66,7 +66,7 @@ public static void main(String[] args) { IDF idf = new IDF().setInputCol("rawFeatures").setOutputCol("features"); IDFModel idfModel = idf.fit(featurizedData); Dataset rescaledData = idfModel.transform(featurizedData); - for (Row r : rescaledData.select("features", "label").takeRows(3)) { + for (Row r : rescaledData.select("features", "label").takeAsList(3)) { Vector features = r.getAs(0); Double label = r.getDouble(1); System.out.println(features); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java index a2f8c436e32f6..9225fe2262f57 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java @@ -59,7 +59,7 @@ public static void main(String[] args) { Tokenizer tokenizer = new Tokenizer().setInputCol("sentence").setOutputCol("words"); Dataset wordsDataFrame = tokenizer.transform(sentenceDataFrame); - for (Row r : wordsDataFrame.select("words", "label").takeRows(3)) { + for (Row r : wordsDataFrame.select("words", "label").takeAsList(3)) { java.util.List words = r.getList(0); for (String word : words) System.out.print(word + " "); System.out.println(); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java index 2dce8c2168c2d..c5bb1eaaa3446 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaWord2VecExample.java @@ -59,7 +59,7 @@ public static void main(String[] args) { .setMinCount(0); Word2VecModel model = word2Vec.fit(documentDF); Dataset result = model.transform(documentDF); - for (Row r : result.select("result").takeRows(3)) { + for (Row r : result.select("result").takeAsList(3)) { System.out.println(r); } // $example off$ diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java index b87605ebfd6a3..e2da11183b93f 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorSlicerSuite.java @@ -78,7 +78,7 @@ public void vectorSlice() { Dataset output = vectorSlicer.transform(dataset); - for (Row r : output.select("userFeatures", "features").takeRows(2)) { + for (Row r : output.select("userFeatures", "features").takeAsList(2)) { Vector features = r.getAs(1); Assert.assertEquals(features.size(), 2); } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index f1791e6943bb7..1ea7db0388689 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1762,10 +1762,6 @@ class Dataset[T] private[sql]( */ def take(n: Int): Array[T] = head(n) - def takeRows(n: Int): Array[Row] = withTypedCallback("takeRows", limit(n)) { ds => - ds.collectRows(needCallback = false) - } - /** * Returns the first `n` rows in the [[DataFrame]] as a list. * @@ -1790,8 +1786,6 @@ class Dataset[T] private[sql]( */ def collect(): Array[T] = collect(needCallback = true) - def collectRows(): Array[Row] = collectRows(needCallback = true) - /** * Returns a Java list that contains all of [[Row]]s in this [[DataFrame]]. * @@ -1820,18 +1814,6 @@ class Dataset[T] private[sql]( } } - private def collectRows(needCallback: Boolean): Array[Row] = { - def execute(): Array[Row] = withNewExecutionId { - queryExecution.executedPlan.executeCollectPublic() - } - - if (needCallback) { - withCallback("collect", toDF())(_ => execute()) - } else { - execute() - } - } - /** * Returns the number of rows in the [[DataFrame]]. * @group action diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java index 42af813bc1cd3..ae9c8cc1ba9ff 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaApplySchemaSuite.java @@ -109,13 +109,13 @@ public Row call(Person person) throws Exception { Dataset df = sqlContext.createDataFrame(rowRDD, schema); df.registerTempTable("people"); - Row[] actual = sqlContext.sql("SELECT * FROM people").collectRows(); + List actual = sqlContext.sql("SELECT * FROM people").collectAsList(); List expected = new ArrayList<>(2); expected.add(RowFactory.create("Michael", 29)); expected.add(RowFactory.create("Yin", 28)); - Assert.assertEquals(expected, Arrays.asList(actual)); + Assert.assertEquals(expected, actual); } @Test diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index 47cc74dbc1f28..42554720edae5 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -19,6 +19,7 @@ import java.io.Serializable; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -65,7 +66,7 @@ public void tearDown() { @Test public void testExecution() { Dataset df = context.table("testData").filter("key = 1"); - Assert.assertEquals(1, df.select("key").collectRows()[0].get(0)); + Assert.assertEquals(1, df.select("key").collectAsList().get(0).get(0)); } @Test @@ -208,8 +209,8 @@ public void testCreateDataFromFromList() { StructType schema = createStructType(Arrays.asList(createStructField("i", IntegerType, true))); List rows = Arrays.asList(RowFactory.create(0)); Dataset df = context.createDataFrame(rows, schema); - Row[] result = df.collectRows(); - Assert.assertEquals(1, result.length); + List result = df.collectAsList(); + Assert.assertEquals(1, result.size()); } @Test @@ -241,8 +242,8 @@ public void testCrosstab() { Assert.assertEquals("a_b", columnNames[0]); Assert.assertEquals("2", columnNames[1]); Assert.assertEquals("1", columnNames[2]); - Row[] rows = crosstab.collectRows(); - Arrays.sort(rows, crosstabRowComparator); + List rows = crosstab.collectAsList(); + Collections.sort(rows, crosstabRowComparator); Integer count = 1; for (Row row : rows) { Assert.assertEquals(row.get(0).toString(), count.toString()); @@ -257,7 +258,7 @@ public void testFrequentItems() { Dataset df = context.table("testData2"); String[] cols = {"a"}; Dataset results = df.stat().freqItems(cols, 0.2); - Assert.assertTrue(results.collectRows()[0].getSeq(0).contains(1)); + Assert.assertTrue(results.collectAsList().get(0).getSeq(0).contains(1)); } @Test @@ -278,27 +279,27 @@ public void testCovariance() { public void testSampleBy() { Dataset df = context.range(0, 100, 1, 2).select(col("id").mod(3).as("key")); Dataset sampled = df.stat().sampleBy("key", ImmutableMap.of(0, 0.1, 1, 0.2), 0L); - Row[] actual = sampled.groupBy("key").count().orderBy("key").collectRows(); - Assert.assertEquals(0, actual[0].getLong(0)); - Assert.assertTrue(0 <= actual[0].getLong(1) && actual[0].getLong(1) <= 8); - Assert.assertEquals(1, actual[1].getLong(0)); - Assert.assertTrue(2 <= actual[1].getLong(1) && actual[1].getLong(1) <= 13); + List actual = sampled.groupBy("key").count().orderBy("key").collectAsList(); + Assert.assertEquals(0, actual.get(0).getLong(0)); + Assert.assertTrue(0 <= actual.get(0).getLong(1) && actual.get(0).getLong(1) <= 8); + Assert.assertEquals(1, actual.get(1).getLong(0)); + Assert.assertTrue(2 <= actual.get(1).getLong(1) && actual.get(1).getLong(1) <= 13); } @Test public void pivot() { Dataset df = context.table("courseSales"); - Row[] actual = df.groupBy("year") + List actual = df.groupBy("year") .pivot("course", Arrays.asList("dotNET", "Java")) - .agg(sum("earnings")).orderBy("year").collectRows(); + .agg(sum("earnings")).orderBy("year").collectAsList(); - Assert.assertEquals(2012, actual[0].getInt(0)); - Assert.assertEquals(15000.0, actual[0].getDouble(1), 0.01); - Assert.assertEquals(20000.0, actual[0].getDouble(2), 0.01); + Assert.assertEquals(2012, actual.get(0).getInt(0)); + Assert.assertEquals(15000.0, actual.get(0).getDouble(1), 0.01); + Assert.assertEquals(20000.0, actual.get(0).getDouble(2), 0.01); - Assert.assertEquals(2013, actual[1].getInt(0)); - Assert.assertEquals(48000.0, actual[1].getDouble(1), 0.01); - Assert.assertEquals(30000.0, actual[1].getDouble(2), 0.01); + Assert.assertEquals(2013, actual.get(1).getInt(0)); + Assert.assertEquals(48000.0, actual.get(1).getDouble(1), 0.01); + Assert.assertEquals(30000.0, actual.get(1).getDouble(2), 0.01); } @Test From db88d0204e3a9a05dbe6e67e1abb942639c50a06 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 13 Mar 2016 12:11:18 +0800 Subject: [PATCH 11/21] [MINOR][DOCS] Replace `DataFrame` with `Dataset` in Javadoc. ## What changes were proposed in this pull request? SPARK-13817 (PR #11656) replaces `DataFrame` with `Dataset` from Java. This PR fixes the remaining broken links and sample Java code in `package-info.java`. As a result, it will update the following Javadoc. * http://spark.apache.org/docs/latest/api/java/org/apache/spark/ml/attribute/package-summary.html * http://spark.apache.org/docs/latest/api/java/org/apache/spark/ml/feature/package-summary.html ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #11675 from dongjoon-hyun/replace_dataframe_with_dataset_in_javadoc. --- .../org/apache/spark/ml/attribute/package-info.java | 2 +- .../org/apache/spark/ml/feature/package-info.java | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java b/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java index e3474f3c1d3ff..464ed125695d1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/package-info.java @@ -20,7 +20,7 @@ /** *

ML attributes

* - * The ML pipeline API uses {@link org.apache.spark.sql.DataFrame}s as ML datasets. + * The ML pipeline API uses {@link org.apache.spark.sql.Dataset}s as ML datasets. * Each dataset consists of typed columns, e.g., string, double, vector, etc. * However, knowing only the column type may not be sufficient to handle the data properly. * For instance, a double column with values 0.0, 1.0, 2.0, ... may represent some label indices, diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/package-info.java b/mllib/src/main/scala/org/apache/spark/ml/feature/package-info.java index 7a35f2d448f9d..dcff4245d1d26 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/package-info.java +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/package-info.java @@ -22,7 +22,7 @@ * The `ml.feature` package provides common feature transformers that help convert raw data or * features into more suitable forms for model fitting. * Most feature transformers are implemented as {@link org.apache.spark.ml.Transformer}s, which - * transforms one {@link org.apache.spark.sql.DataFrame} into another, e.g., + * transforms one {@link org.apache.spark.sql.Dataset} into another, e.g., * {@link org.apache.spark.ml.feature.HashingTF}. * Some feature transformers are implemented as {@link org.apache.spark.ml.Estimator}}s, because the * transformation requires some aggregated information of the dataset, e.g., document @@ -31,7 +31,7 @@ * obtain the model first, e.g., {@link org.apache.spark.ml.feature.IDFModel}, in order to apply * transformation. * The transformation is usually done by appending new columns to the input - * {@link org.apache.spark.sql.DataFrame}, so all input columns are carried over. + * {@link org.apache.spark.sql.Dataset}, so all input columns are carried over. * * We try to make each transformer minimal, so it becomes flexible to assemble feature * transformation pipelines. @@ -46,7 +46,7 @@ * import org.apache.spark.api.java.JavaRDD; * import static org.apache.spark.sql.types.DataTypes.*; * import org.apache.spark.sql.types.StructType; - * import org.apache.spark.sql.DataFrame; + * import org.apache.spark.sql.Dataset; * import org.apache.spark.sql.RowFactory; * import org.apache.spark.sql.Row; * @@ -66,7 +66,7 @@ * RowFactory.create(0, "Hi I heard about Spark", 3.0), * RowFactory.create(1, "I wish Java could use case classes", 4.0), * RowFactory.create(2, "Logistic regression models are neat", 4.0))); - * DataFrame df = jsql.createDataFrame(rowRDD, schema); + * Dataset dataset = jsql.createDataFrame(rowRDD, schema); * // define feature transformers * RegexTokenizer tok = new RegexTokenizer() * .setInputCol("text") @@ -88,10 +88,10 @@ * // assemble and fit the feature transformation pipeline * Pipeline pipeline = new Pipeline() * .setStages(new PipelineStage[] {tok, sw, tf, idf, assembler}); - * PipelineModel model = pipeline.fit(df); + * PipelineModel model = pipeline.fit(dataset); * * // save transformed features with raw data - * model.transform(df) + * model.transform(dataset) * .select("id", "text", "rating", "features") * .write().format("parquet").save("/output/path"); * From 515e4afbc7ec957609451ea75772d6ef1b914908 Mon Sep 17 00:00:00 2001 From: Bjorn Jonsson Date: Sun, 13 Mar 2016 10:18:24 +0000 Subject: [PATCH 12/21] [SPARK-13810][CORE] Add Port Configuration Suggestions on Bind Exceptions ## What changes were proposed in this pull request? Currently, when a java.net.BindException is thrown, it displays the following message: java.net.BindException: Address already in use: Service '$serviceName' failed after 16 retries! This change adds port configuration suggestions to the BindException, for example, for the UI, it now displays java.net.BindException: Address already in use: Service 'SparkUI' failed after 16 retries! Consider explicitly setting the appropriate port for 'SparkUI' (for example spark.ui.port for SparkUI) to an available port or increasing spark.port.maxRetries. ## How was this patch tested? Manual tests Author: Bjorn Jonsson Closes #11644 from bjornjon/master. --- core/src/main/scala/org/apache/spark/util/Utils.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 9688cca4f0fcd..b4c49513711c2 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2014,8 +2014,10 @@ private[spark] object Utils extends Logging { } catch { case e: Exception if isBindCollision(e) => if (offset >= maxRetries) { - val exceptionMessage = - s"${e.getMessage}: Service$serviceString failed after $maxRetries retries!" + val exceptionMessage = s"${e.getMessage}: Service$serviceString failed after " + + s"$maxRetries retries! Consider explicitly setting the appropriate port for the " + + s"service$serviceString (for example spark.ui.port for SparkUI) to an available " + + "port or increasing spark.port.maxRetries." val exception = new BindException(exceptionMessage) // restore original stack trace exception.setStackTrace(e.getStackTrace) From c7e68c3968357268f705dee1477c448472c21547 Mon Sep 17 00:00:00 2001 From: Sun Rui Date: Sun, 13 Mar 2016 14:30:44 -0700 Subject: [PATCH 13/21] [SPARK-13812][SPARKR] Fix SparkR lint-r test errors. ## What changes were proposed in this pull request? This PR fixes all newly captured SparkR lint-r errors after the lintr package is updated from github. ## How was this patch tested? dev/lint-r SparkR unit tests Author: Sun Rui Closes #11652 from sun-rui/SPARK-13812. --- R/pkg/DESCRIPTION | 5 +- R/pkg/R/DataFrame.R | 16 +- R/pkg/R/RDD.R | 2 +- R/pkg/R/context.R | 3 + R/pkg/R/deserialize.R | 2 +- R/pkg/R/generics.R | 4 +- R/pkg/R/mllib.R | 2 +- R/pkg/R/serialize.R | 2 +- R/pkg/R/sparkR.R | 6 +- R/pkg/R/utils.R | 4 +- R/pkg/inst/profile/general.R | 2 +- R/pkg/inst/tests/testthat/packageInAJarTest.R | 4 +- R/pkg/inst/tests/testthat/test_binaryFile.R | 14 +- .../tests/testthat/test_binary_function.R | 6 +- R/pkg/inst/tests/testthat/test_broadcast.R | 4 +- R/pkg/inst/tests/testthat/test_mllib.R | 4 +- R/pkg/inst/tests/testthat/test_rdd.R | 82 ++++----- R/pkg/inst/tests/testthat/test_sparkSQL.R | 156 +++++++++--------- R/pkg/inst/tests/testthat/test_textFile.R | 24 +-- R/pkg/inst/tests/testthat/test_utils.R | 8 +- R/pkg/inst/worker/worker.R | 2 +- 21 files changed, 178 insertions(+), 174 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 465bc37788e5d..0cd0d75df0f70 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -18,10 +18,10 @@ Collate: 'schema.R' 'generics.R' 'jobj.R' - 'RDD.R' - 'pairRDD.R' 'column.R' 'group.R' + 'RDD.R' + 'pairRDD.R' 'DataFrame.R' 'SQLContext.R' 'backend.R' @@ -36,3 +36,4 @@ Collate: 'stats.R' 'types.R' 'utils.R' +RoxygenNote: 5.0.1 diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 50655e9382325..a64a013b654ef 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -321,7 +321,7 @@ setMethod("colnames<-", } # Check if the column names have . in it - if (any(regexec(".", value, fixed=TRUE)[[1]][1] != -1)) { + if (any(regexec(".", value, fixed = TRUE)[[1]][1] != -1)) { stop("Colum names cannot contain the '.' symbol.") } @@ -351,7 +351,7 @@ setMethod("coltypes", types <- sapply(dtypes(x), function(x) {x[[2]]}) # Map Spark data types into R's data types using DATA_TYPES environment - rTypes <- sapply(types, USE.NAMES=F, FUN=function(x) { + rTypes <- sapply(types, USE.NAMES = F, FUN = function(x) { # Check for primitive types type <- PRIMITIVE_TYPES[[x]] @@ -1779,7 +1779,7 @@ setMethod("merge", signature(x = "DataFrame", y = "DataFrame"), function(x, y, by = intersect(names(x), names(y)), by.x = by, by.y = by, all = FALSE, all.x = all, all.y = all, - sort = TRUE, suffixes = c("_x","_y"), ... ) { + sort = TRUE, suffixes = c("_x", "_y"), ... ) { if (length(suffixes) != 2) { stop("suffixes must have length 2") @@ -2299,7 +2299,7 @@ setMethod("as.data.frame", function(x, ...) { # Check if additional parameters have been passed if (length(list(...)) > 0) { - stop(paste("Unused argument(s): ", paste(list(...), collapse=", "))) + stop(paste("Unused argument(s): ", paste(list(...), collapse = ", "))) } collect(x) }) @@ -2395,13 +2395,13 @@ setMethod("str", # Get the first elements for each column firstElements <- if (types[i] == "character") { - paste(paste0("\"", localDF[,i], "\""), collapse = " ") + paste(paste0("\"", localDF[, i], "\""), collapse = " ") } else { - paste(localDF[,i], collapse = " ") + paste(localDF[, i], collapse = " ") } # Add the corresponding number of spaces for alignment - spaces <- paste(rep(" ", max(nchar(names) - nchar(names[i]))), collapse="") + spaces <- paste(rep(" ", max(nchar(names) - nchar(names[i]))), collapse = "") # Get the short type. For 'character', it would be 'chr'; # 'for numeric', it's 'num', etc. @@ -2413,7 +2413,7 @@ setMethod("str", # Concatenate the colnames, coltypes, and first # elements of each column line <- paste0(" $ ", names[i], spaces, ": ", - dataType, " ",firstElements) + dataType, " ", firstElements) # Chop off extra characters if this is too long cat(substr(line, 1, MAX_CHAR_PER_ROW)) diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index a78fbb714f2be..35c4e6f1afaf4 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -67,7 +67,7 @@ setMethod("initialize", "RDD", function(.Object, jrdd, serializedMode, setMethod("show", "RDD", function(object) { - cat(paste(callJMethod(getJRDD(object), "toString"), "\n", sep="")) + cat(paste(callJMethod(getJRDD(object), "toString"), "\n", sep = "")) }) setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) { diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 471bec1eacf03..b0e67c8ad26ab 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -103,7 +103,10 @@ parallelize <- function(sc, coll, numSlices = 1) { # TODO: bound/safeguard numSlices # TODO: unit tests for if the split works for all primitives # TODO: support matrix, data frame, etc + # nolint start + # suppress lintr warning: Place a space before left parenthesis, except in a function call. if ((!is.list(coll) && !is.vector(coll)) || is.data.frame(coll)) { + # nolint end if (is.data.frame(coll)) { message(paste("context.R: A data frame is parallelized by columns.")) } else { diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index d8a0393275390..eefdf178733fd 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -186,7 +186,7 @@ readMultipleObjects <- function(inputCon) { # of the objects, so the number of objects varies, we try to read # all objects in a loop until the end of the stream. data <- list() - while(TRUE) { + while (TRUE) { # If reaching the end of the stream, type returned should be "". type <- readType(inputCon) if (type == "") { diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index ddfa61717af2e..6ad71fcb46712 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -607,7 +607,7 @@ setGeneric("selectExpr", function(x, expr, ...) { standardGeneric("selectExpr") #' @rdname showDF #' @export -setGeneric("showDF", function(x,...) { standardGeneric("showDF") }) +setGeneric("showDF", function(x, ...) { standardGeneric("showDF") }) # @rdname subset # @export @@ -615,7 +615,7 @@ setGeneric("subset", function(x, ...) { standardGeneric("subset") }) #' @rdname agg #' @export -setGeneric("summarize", function(x,...) { standardGeneric("summarize") }) +setGeneric("summarize", function(x, ...) { standardGeneric("summarize") }) #' @rdname summary #' @export diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index 346f33d7dab2c..5c0d3dcf3af90 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -54,7 +54,7 @@ setMethod("glm", signature(formula = "formula", family = "ANY", data = "DataFram function(formula, family = c("gaussian", "binomial"), data, lambda = 0, alpha = 0, standardize = TRUE, solver = "auto") { family <- match.arg(family) - formula <- paste(deparse(formula), collapse="") + formula <- paste(deparse(formula), collapse = "") model <- callJStatic("org.apache.spark.ml.api.r.SparkRWrappers", "fitRModelFormula", formula, data@sdf, family, lambda, alpha, standardize, solver) diff --git a/R/pkg/R/serialize.R b/R/pkg/R/serialize.R index 70e87a93e610f..3bbf60d9b668c 100644 --- a/R/pkg/R/serialize.R +++ b/R/pkg/R/serialize.R @@ -100,7 +100,7 @@ writeJobj <- function(con, value) { writeString <- function(con, value) { utfVal <- enc2utf8(value) writeInt(con, as.integer(nchar(utfVal, type = "bytes") + 1)) - writeBin(utfVal, con, endian = "big", useBytes=TRUE) + writeBin(utfVal, con, endian = "big", useBytes = TRUE) } writeInt <- function(con, value) { diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 3e9eafc7f5b90..c187869fdf121 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -153,7 +153,7 @@ sparkR.init <- function( if (!file.exists(path)) { stop("JVM is not ready after 10 seconds") } - f <- file(path, open="rb") + f <- file(path, open = "rb") backendPort <- readInt(f) monitorPort <- readInt(f) rLibPath <- readString(f) @@ -185,9 +185,9 @@ sparkR.init <- function( } sparkExecutorEnvMap <- convertNamedListToEnv(sparkExecutorEnv) - if(is.null(sparkExecutorEnvMap$LD_LIBRARY_PATH)) { + if (is.null(sparkExecutorEnvMap$LD_LIBRARY_PATH)) { sparkExecutorEnvMap[["LD_LIBRARY_PATH"]] <- - paste0("$LD_LIBRARY_PATH:",Sys.getenv("LD_LIBRARY_PATH")) + paste0("$LD_LIBRARY_PATH:", Sys.getenv("LD_LIBRARY_PATH")) } # Classpath separator is ";" on Windows diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index aa386e5da933b..fb6575cb42907 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -158,7 +158,7 @@ wrapInt <- function(value) { # Multiply `val` by 31 and add `addVal` to the result. Ensures that # integer-overflows are handled at every step. mult31AndAdd <- function(val, addVal) { - vec <- c(bitwShiftL(val, c(4,3,2,1,0)), addVal) + vec <- c(bitwShiftL(val, c(4, 3, 2, 1, 0)), addVal) Reduce(function(a, b) { wrapInt(as.numeric(a) + as.numeric(b)) }, @@ -202,7 +202,7 @@ serializeToString <- function(rdd) { # This function amortizes the allocation cost by doubling # the size of the list every time it fills up. addItemToAccumulator <- function(acc, item) { - if(acc$counter == acc$size) { + if (acc$counter == acc$size) { acc$size <- acc$size * 2 length(acc$data) <- acc$size } diff --git a/R/pkg/inst/profile/general.R b/R/pkg/inst/profile/general.R index c55fe9ba7af7a..8c75c19ca7ac3 100644 --- a/R/pkg/inst/profile/general.R +++ b/R/pkg/inst/profile/general.R @@ -19,5 +19,5 @@ packageDir <- Sys.getenv("SPARKR_PACKAGE_DIR") dirs <- strsplit(packageDir, ",")[[1]] .libPaths(c(dirs, .libPaths())) - Sys.setenv(NOAWT=1) + Sys.setenv(NOAWT = 1) } diff --git a/R/pkg/inst/tests/testthat/packageInAJarTest.R b/R/pkg/inst/tests/testthat/packageInAJarTest.R index 207a37a0cb47f..c26b28b78dee8 100644 --- a/R/pkg/inst/tests/testthat/packageInAJarTest.R +++ b/R/pkg/inst/tests/testthat/packageInAJarTest.R @@ -25,6 +25,6 @@ run2 <- myfunc(-4L) sparkR.stop() -if(run1 != 6) quit(save = "no", status = 1) +if (run1 != 6) quit(save = "no", status = 1) -if(run2 != -3) quit(save = "no", status = 1) +if (run2 != -3) quit(save = "no", status = 1) diff --git a/R/pkg/inst/tests/testthat/test_binaryFile.R b/R/pkg/inst/tests/testthat/test_binaryFile.R index f2452ed97d2ea..976a7558a816d 100644 --- a/R/pkg/inst/tests/testthat/test_binaryFile.R +++ b/R/pkg/inst/tests/testthat/test_binaryFile.R @@ -23,8 +23,8 @@ sc <- sparkR.init() mockFile <- c("Spark is pretty.", "Spark is awesome.") test_that("saveAsObjectFile()/objectFile() following textFile() works", { - fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") - fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName1 <- tempfile(pattern = "spark-test", fileext = ".tmp") + fileName2 <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName1) rdd <- textFile(sc, fileName1, 1) @@ -37,7 +37,7 @@ test_that("saveAsObjectFile()/objectFile() following textFile() works", { }) test_that("saveAsObjectFile()/objectFile() works on a parallelized list", { - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") l <- list(1, 2, 3) rdd <- parallelize(sc, l, 1) @@ -49,8 +49,8 @@ test_that("saveAsObjectFile()/objectFile() works on a parallelized list", { }) test_that("saveAsObjectFile()/objectFile() following RDD transformations works", { - fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") - fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName1 <- tempfile(pattern = "spark-test", fileext = ".tmp") + fileName2 <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName1) rdd <- textFile(sc, fileName1) @@ -73,8 +73,8 @@ test_that("saveAsObjectFile()/objectFile() following RDD transformations works", }) test_that("saveAsObjectFile()/objectFile() works with multiple paths", { - fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") - fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName1 <- tempfile(pattern = "spark-test", fileext = ".tmp") + fileName2 <- tempfile(pattern = "spark-test", fileext = ".tmp") rdd1 <- parallelize(sc, "Spark is pretty.") saveAsObjectFile(rdd1, fileName1) diff --git a/R/pkg/inst/tests/testthat/test_binary_function.R b/R/pkg/inst/tests/testthat/test_binary_function.R index f054ac9a87d61..7bad4d2a7e106 100644 --- a/R/pkg/inst/tests/testthat/test_binary_function.R +++ b/R/pkg/inst/tests/testthat/test_binary_function.R @@ -31,7 +31,7 @@ test_that("union on two RDDs", { actual <- collect(unionRDD(rdd, rdd)) expect_equal(actual, as.list(rep(nums, 2))) - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName) text.rdd <- textFile(sc, fileName) @@ -74,10 +74,10 @@ test_that("zipPartitions() on RDDs", { actual <- collect(zipPartitions(rdd1, rdd2, rdd3, func = function(x, y, z) { list(list(x, y, z))} )) expect_equal(actual, - list(list(1, c(1,2), c(1,2,3)), list(2, c(3,4), c(4,5,6)))) + list(list(1, c(1, 2), c(1, 2, 3)), list(2, c(3, 4), c(4, 5, 6)))) mockFile <- c("Spark is pretty.", "Spark is awesome.") - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName) rdd <- textFile(sc, fileName, 1) diff --git a/R/pkg/inst/tests/testthat/test_broadcast.R b/R/pkg/inst/tests/testthat/test_broadcast.R index bb86a5c922bde..8be6efc3dbed3 100644 --- a/R/pkg/inst/tests/testthat/test_broadcast.R +++ b/R/pkg/inst/tests/testthat/test_broadcast.R @@ -25,7 +25,7 @@ nums <- 1:2 rrdd <- parallelize(sc, nums, 2L) test_that("using broadcast variable", { - randomMat <- matrix(nrow=10, ncol=10, data=rnorm(100)) + randomMat <- matrix(nrow = 10, ncol = 10, data = rnorm(100)) randomMatBr <- broadcast(sc, randomMat) useBroadcast <- function(x) { @@ -37,7 +37,7 @@ test_that("using broadcast variable", { }) test_that("without using broadcast variable", { - randomMat <- matrix(nrow=10, ncol=10, data=rnorm(100)) + randomMat <- matrix(nrow = 10, ncol = 10, data = rnorm(100)) useBroadcast <- function(x) { sum(randomMat * x) diff --git a/R/pkg/inst/tests/testthat/test_mllib.R b/R/pkg/inst/tests/testthat/test_mllib.R index af84a0abcf94d..e120462964d1e 100644 --- a/R/pkg/inst/tests/testthat/test_mllib.R +++ b/R/pkg/inst/tests/testthat/test_mllib.R @@ -96,9 +96,9 @@ test_that("summary coefficients match with native glm of family 'binomial'", { training <- filter(df, df$Species != "setosa") stats <- summary(glm(Species ~ Sepal_Length + Sepal_Width, data = training, family = "binomial")) - coefs <- as.vector(stats$coefficients[,1]) + coefs <- as.vector(stats$coefficients[, 1]) - rTraining <- iris[iris$Species %in% c("versicolor","virginica"),] + rTraining <- iris[iris$Species %in% c("versicolor", "virginica"), ] rCoefs <- as.vector(coef(glm(Species ~ Sepal.Length + Sepal.Width, data = rTraining, family = binomial(link = "logit")))) diff --git a/R/pkg/inst/tests/testthat/test_rdd.R b/R/pkg/inst/tests/testthat/test_rdd.R index 1b3a22486e95f..3b0c16be5a754 100644 --- a/R/pkg/inst/tests/testthat/test_rdd.R +++ b/R/pkg/inst/tests/testthat/test_rdd.R @@ -75,7 +75,7 @@ test_that("mapPartitions on RDD", { test_that("flatMap() on RDDs", { flat <- flatMap(intRdd, function(x) { list(x, x) }) actual <- collect(flat) - expect_equal(actual, rep(intPairs, each=2)) + expect_equal(actual, rep(intPairs, each = 2)) }) test_that("filterRDD on RDD", { @@ -245,9 +245,9 @@ test_that("mapValues() on pairwise RDDs", { }) test_that("flatMapValues() on pairwise RDDs", { - l <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) + l <- parallelize(sc, list(list(1, c(1, 2)), list(2, c(3, 4)))) actual <- collect(flatMapValues(l, function(x) { x })) - expect_equal(actual, list(list(1,1), list(1,2), list(2,3), list(2,4))) + expect_equal(actual, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) # Generate x to x+1 for every value actual <- collect(flatMapValues(intRdd, function(x) { x: (x + 1) })) @@ -448,12 +448,12 @@ test_that("zipRDD() on RDDs", { list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004))) mockFile <- c("Spark is pretty.", "Spark is awesome.") - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName) rdd <- textFile(sc, fileName, 1) actual <- collect(zipRDD(rdd, rdd)) - expected <- lapply(mockFile, function(x) { list(x ,x) }) + expected <- lapply(mockFile, function(x) { list(x, x) }) expect_equal(actual, expected) rdd1 <- parallelize(sc, 0:1, 1) @@ -484,7 +484,7 @@ test_that("cartesian() on RDDs", { expect_equal(actual, list()) mockFile <- c("Spark is pretty.", "Spark is awesome.") - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName) rdd <- textFile(sc, fileName) @@ -523,19 +523,19 @@ test_that("subtract() on RDDs", { # subtract by an empty RDD rdd2 <- parallelize(sc, list()) actual <- collect(subtract(rdd1, rdd2)) - expect_equal(as.list(sort(as.vector(actual, mode="integer"))), + expect_equal(as.list(sort(as.vector(actual, mode = "integer"))), l) rdd2 <- parallelize(sc, list(2, 4)) actual <- collect(subtract(rdd1, rdd2)) - expect_equal(as.list(sort(as.vector(actual, mode="integer"))), + expect_equal(as.list(sort(as.vector(actual, mode = "integer"))), list(1, 1, 3)) l <- list("a", "a", "b", "b", "c", "d") rdd1 <- parallelize(sc, l) rdd2 <- parallelize(sc, list("b", "d")) actual <- collect(subtract(rdd1, rdd2)) - expect_equal(as.list(sort(as.vector(actual, mode="character"))), + expect_equal(as.list(sort(as.vector(actual, mode = "character"))), list("a", "a", "c")) }) @@ -585,53 +585,53 @@ test_that("intersection() on RDDs", { }) test_that("join() on pairwise RDDs", { - rdd1 <- parallelize(sc, list(list(1,1), list(2,4))) - rdd2 <- parallelize(sc, list(list(1,2), list(1,3))) + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) + rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) actual <- collect(join(rdd1, rdd2, 2L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(list(list(1, list(1, 2)), list(1, list(1, 3))))) - rdd1 <- parallelize(sc, list(list("a",1), list("b",4))) - rdd2 <- parallelize(sc, list(list("a",2), list("a",3))) + rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4))) + rdd2 <- parallelize(sc, list(list("a", 2), list("a", 3))) actual <- collect(join(rdd1, rdd2, 2L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(list(list("a", list(1, 2)), list("a", list(1, 3))))) - rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) - rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 2))) + rdd2 <- parallelize(sc, list(list(3, 3), list(4, 4))) actual <- collect(join(rdd1, rdd2, 2L)) expect_equal(actual, list()) - rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) - rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + rdd1 <- parallelize(sc, list(list("a", 1), list("b", 2))) + rdd2 <- parallelize(sc, list(list("c", 3), list("d", 4))) actual <- collect(join(rdd1, rdd2, 2L)) expect_equal(actual, list()) }) test_that("leftOuterJoin() on pairwise RDDs", { - rdd1 <- parallelize(sc, list(list(1,1), list(2,4))) - rdd2 <- parallelize(sc, list(list(1,2), list(1,3))) + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) + rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) expected <- list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) - rdd1 <- parallelize(sc, list(list("a",1), list("b",4))) - rdd2 <- parallelize(sc, list(list("a",2), list("a",3))) + rdd1 <- parallelize(sc, list(list("a", 1), list("b", 4))) + rdd2 <- parallelize(sc, list(list("a", 2), list("a", 3))) actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) expected <- list(list("b", list(4, NULL)), list("a", list(1, 2)), list("a", list(1, 3))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) - rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) - rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 2))) + rdd2 <- parallelize(sc, list(list(3, 3), list(4, 4))) actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) expected <- list(list(1, list(1, NULL)), list(2, list(2, NULL))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) - rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) - rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + rdd1 <- parallelize(sc, list(list("a", 1), list("b", 2))) + rdd2 <- parallelize(sc, list(list("c", 3), list("d", 4))) actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) expected <- list(list("b", list(2, NULL)), list("a", list(1, NULL))) expect_equal(sortKeyValueList(actual), @@ -639,57 +639,57 @@ test_that("leftOuterJoin() on pairwise RDDs", { }) test_that("rightOuterJoin() on pairwise RDDs", { - rdd1 <- parallelize(sc, list(list(1,2), list(1,3))) - rdd2 <- parallelize(sc, list(list(1,1), list(2,4))) + rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3))) + rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) - rdd1 <- parallelize(sc, list(list("a",2), list("a",3))) - rdd2 <- parallelize(sc, list(list("a",1), list("b",4))) + rdd1 <- parallelize(sc, list(list("a", 2), list("a", 3))) + rdd2 <- parallelize(sc, list(list("a", 1), list("b", 4))) actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), list("a", list(3, 1))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) - rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) - rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 2))) + rdd2 <- parallelize(sc, list(list(3, 3), list(4, 4))) actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(list(list(3, list(NULL, 3)), list(4, list(NULL, 4))))) - rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) - rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + rdd1 <- parallelize(sc, list(list("a", 1), list("b", 2))) + rdd2 <- parallelize(sc, list(list("c", 3), list("d", 4))) actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(list(list("d", list(NULL, 4)), list("c", list(NULL, 3))))) }) test_that("fullOuterJoin() on pairwise RDDs", { - rdd1 <- parallelize(sc, list(list(1,2), list(1,3), list(3,3))) - rdd2 <- parallelize(sc, list(list(1,1), list(2,4))) + rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3), list(3, 3))) + rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4)), list(3, list(3, NULL))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) - rdd1 <- parallelize(sc, list(list("a",2), list("a",3), list("c", 1))) - rdd2 <- parallelize(sc, list(list("a",1), list("b",4))) + rdd1 <- parallelize(sc, list(list("a", 2), list("a", 3), list("c", 1))) + rdd2 <- parallelize(sc, list(list("a", 1), list("b", 4))) actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), list("a", list(3, 1)), list("c", list(1, NULL))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) - rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) - rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 2))) + rdd2 <- parallelize(sc, list(list(3, 3), list(4, 4))) actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(list(list(1, list(1, NULL)), list(2, list(2, NULL)), list(3, list(NULL, 3)), list(4, list(NULL, 4))))) - rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) - rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + rdd1 <- parallelize(sc, list(list("a", 1), list("b", 2))) + rdd2 <- parallelize(sc, list(list("c", 3), list("d", 4))) actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) expect_equal(sortKeyValueList(actual), sortKeyValueList(list(list("a", list(1, NULL)), list("b", list(2, NULL)), diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 11a8f12fd5432..63acbadfa6a16 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -41,8 +41,8 @@ sqlContext <- sparkRSQL.init(sc) mockLines <- c("{\"name\":\"Michael\"}", "{\"name\":\"Andy\", \"age\":30}", "{\"name\":\"Justin\", \"age\":19}") -jsonPath <- tempfile(pattern="sparkr-test", fileext=".tmp") -parquetPath <- tempfile(pattern="sparkr-test", fileext=".parquet") +jsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") +parquetPath <- tempfile(pattern = "sparkr-test", fileext = ".parquet") writeLines(mockLines, jsonPath) # For test nafunctions, like dropna(), fillna(),... @@ -51,7 +51,7 @@ mockLinesNa <- c("{\"name\":\"Bob\",\"age\":16,\"height\":176.5}", "{\"name\":\"David\",\"age\":60,\"height\":null}", "{\"name\":\"Amy\",\"age\":null,\"height\":null}", "{\"name\":null,\"age\":null,\"height\":null}") -jsonPathNa <- tempfile(pattern="sparkr-test", fileext=".tmp") +jsonPathNa <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(mockLinesNa, jsonPathNa) # For test complex types in DataFrame @@ -59,7 +59,7 @@ mockLinesComplexType <- c("{\"c1\":[1, 2, 3], \"c2\":[\"a\", \"b\", \"c\"], \"c3\":[1.0, 2.0, 3.0]}", "{\"c1\":[4, 5, 6], \"c2\":[\"d\", \"e\", \"f\"], \"c3\":[4.0, 5.0, 6.0]}", "{\"c1\":[7, 8, 9], \"c2\":[\"g\", \"h\", \"i\"], \"c3\":[7.0, 8.0, 9.0]}") -complexTypeJsonPath <- tempfile(pattern="sparkr-test", fileext=".tmp") +complexTypeJsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(mockLinesComplexType, complexTypeJsonPath) test_that("calling sparkRSQL.init returns existing SQL context", { @@ -151,9 +151,9 @@ test_that("create DataFrame from RDD", { expect_equal(as.list(collect(where(df2AsDF, df2AsDF$name == "Bob"))), list(name = "Bob", age = 16, height = 176.5)) - localDF <- data.frame(name=c("John", "Smith", "Sarah"), - age=c(19L, 23L, 18L), - height=c(176.5, 181.4, 173.7)) + localDF <- data.frame(name = c("John", "Smith", "Sarah"), + age = c(19L, 23L, 18L), + height = c(176.5, 181.4, 173.7)) df <- createDataFrame(sqlContext, localDF, schema) expect_is(df, "DataFrame") expect_equal(count(df), 3) @@ -263,7 +263,7 @@ test_that("create DataFrame from list or data.frame", { irisdf <- suppressWarnings(createDataFrame(sqlContext, iris)) iris_collected <- collect(irisdf) - expect_equivalent(iris_collected[,-5], iris[,-5]) + expect_equivalent(iris_collected[, -5], iris[, -5]) expect_equal(iris_collected$Species, as.character(iris$Species)) mtcarsdf <- createDataFrame(sqlContext, mtcars) @@ -329,7 +329,7 @@ test_that("create DataFrame from a data.frame with complex types", { mockLinesMapType <- c("{\"name\":\"Bob\",\"info\":{\"age\":16,\"height\":176.5}}", "{\"name\":\"Alice\",\"info\":{\"age\":20,\"height\":164.3}}", "{\"name\":\"David\",\"info\":{\"age\":60,\"height\":180}}") -mapTypeJsonPath <- tempfile(pattern="sparkr-test", fileext=".tmp") +mapTypeJsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(mockLinesMapType, mapTypeJsonPath) test_that("Collect DataFrame with complex types", { @@ -399,11 +399,11 @@ test_that("read/write json files", { expect_equal(count(df), 3) # Test write.df - jsonPath2 <- tempfile(pattern="jsonPath2", fileext=".json") - write.df(df, jsonPath2, "json", mode="overwrite") + jsonPath2 <- tempfile(pattern = "jsonPath2", fileext = ".json") + write.df(df, jsonPath2, "json", mode = "overwrite") # Test write.json - jsonPath3 <- tempfile(pattern="jsonPath3", fileext=".json") + jsonPath3 <- tempfile(pattern = "jsonPath3", fileext = ".json") write.json(df, jsonPath3) # Test read.json()/jsonFile() works with multiple input paths @@ -466,7 +466,7 @@ test_that("insertInto() on a registered table", { lines <- c("{\"name\":\"Bob\", \"age\":24}", "{\"name\":\"James\", \"age\":35}") - jsonPath2 <- tempfile(pattern="jsonPath2", fileext=".tmp") + jsonPath2 <- tempfile(pattern = "jsonPath2", fileext = ".tmp") parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet") writeLines(lines, jsonPath2) df2 <- read.df(sqlContext, jsonPath2, "json") @@ -526,7 +526,7 @@ test_that("union on mixed serialization types correctly returns a byte RRDD", { textLines <- c("Michael", "Andy, 30", "Justin, 19") - textPath <- tempfile(pattern="sparkr-textLines", fileext=".tmp") + textPath <- tempfile(pattern = "sparkr-textLines", fileext = ".tmp") writeLines(textLines, textPath) textRDD <- textFile(sc, textPath) @@ -547,7 +547,7 @@ test_that("union on mixed serialization types correctly returns a byte RRDD", { }) test_that("objectFile() works with row serialization", { - objectPath <- tempfile(pattern="spark-test", fileext=".tmp") + objectPath <- tempfile(pattern = "spark-test", fileext = ".tmp") df <- read.json(sqlContext, jsonPath) dfRDD <- toRDD(df) saveAsObjectFile(coalesce(dfRDD, 1L), objectPath) @@ -611,7 +611,7 @@ test_that("collect() support Unicode characters", { "{\"name\":\"こんにちは\", \"age\":19}", "{\"name\":\"Xin chào\"}") - jsonPath <- tempfile(pattern="sparkr-test", fileext=".tmp") + jsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(lines, jsonPath) df <- read.df(sqlContext, jsonPath, "json") @@ -705,7 +705,7 @@ test_that("names() colnames() set the column names", { # Test base::colnames base::names m2 <- cbind(1, 1:4) expect_equal(colnames(m2, do.NULL = FALSE), c("col1", "col2")) - colnames(m2) <- c("x","Y") + colnames(m2) <- c("x", "Y") expect_equal(colnames(m2), c("x", "Y")) z <- list(a = 1, b = "c", c = 1:3) @@ -745,7 +745,7 @@ test_that("distinct(), unique() and dropDuplicates() on DataFrames", { "{\"name\":\"Andy\", \"age\":30}", "{\"name\":\"Justin\", \"age\":19}", "{\"name\":\"Justin\", \"age\":19}") - jsonPathWithDup <- tempfile(pattern="sparkr-test", fileext=".tmp") + jsonPathWithDup <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(lines, jsonPathWithDup) df <- read.json(sqlContext, jsonPathWithDup) @@ -774,7 +774,7 @@ test_that("distinct(), unique() and dropDuplicates() on DataFrames", { c(2, 2, 1), c(2, 2, 2)) names(expected) <- c("key", "value1", "value2") expect_equivalent( - result[order(result$key, result$value1, result$value2),], + result[order(result$key, result$value1, result$value2), ], expected) result <- collect(dropDuplicates(df, c("key", "value1"))) @@ -782,7 +782,7 @@ test_that("distinct(), unique() and dropDuplicates() on DataFrames", { c(1, 1, 1), c(1, 2, 1), c(2, 1, 2), c(2, 2, 2)) names(expected) <- c("key", "value1", "value2") expect_equivalent( - result[order(result$key, result$value1, result$value2),], + result[order(result$key, result$value1, result$value2), ], expected) result <- collect(dropDuplicates(df, "key")) @@ -790,7 +790,7 @@ test_that("distinct(), unique() and dropDuplicates() on DataFrames", { c(1, 1, 1), c(2, 1, 2)) names(expected) <- c("key", "value1", "value2") expect_equivalent( - result[order(result$key, result$value1, result$value2),], + result[order(result$key, result$value1, result$value2), ], expected) }) @@ -822,10 +822,10 @@ test_that("select operators", { expect_is(df[[2]], "Column") expect_is(df[["age"]], "Column") - expect_is(df[,1], "DataFrame") - expect_equal(columns(df[,1]), c("name")) - expect_equal(columns(df[,"age"]), c("age")) - df2 <- df[,c("age", "name")] + expect_is(df[, 1], "DataFrame") + expect_equal(columns(df[, 1]), c("name")) + expect_equal(columns(df[, "age"]), c("age")) + df2 <- df[, c("age", "name")] expect_is(df2, "DataFrame") expect_equal(columns(df2), c("age", "name")) @@ -884,7 +884,7 @@ test_that("drop column", { test_that("subsetting", { # read.json returns columns in random order df <- select(read.json(sqlContext, jsonPath), "name", "age") - filtered <- df[df$age > 20,] + filtered <- df[df$age > 20, ] expect_equal(count(filtered), 1) expect_equal(columns(filtered), c("name", "age")) expect_equal(collect(filtered)$name, "Andy") @@ -903,11 +903,11 @@ test_that("subsetting", { expect_equal(count(df4), 2) expect_equal(columns(df4), c("name", "age")) - df5 <- df[df$age %in% c(19), c(1,2)] + df5 <- df[df$age %in% c(19), c(1, 2)] expect_equal(count(df5), 1) expect_equal(columns(df5), c("name", "age")) - df6 <- subset(df, df$age %in% c(30), c(1,2)) + df6 <- subset(df, df$age %in% c(30), c(1, 2)) expect_equal(count(df6), 1) expect_equal(columns(df6), c("name", "age")) @@ -959,22 +959,22 @@ test_that("test HiveContext", { expect_is(df2, "DataFrame") expect_equal(count(df2), 3) - jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") + jsonPath2 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") invisible(saveAsTable(df, "json2", "json", "append", path = jsonPath2)) df3 <- sql(hiveCtx, "select * from json2") expect_is(df3, "DataFrame") expect_equal(count(df3), 3) unlink(jsonPath2) - hivetestDataPath <- tempfile(pattern="sparkr-test", fileext=".tmp") + hivetestDataPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") invisible(saveAsTable(df, "hivetestbl", path = hivetestDataPath)) df4 <- sql(hiveCtx, "select * from hivetestbl") expect_is(df4, "DataFrame") expect_equal(count(df4), 3) unlink(hivetestDataPath) - parquetDataPath <- tempfile(pattern="sparkr-test", fileext=".tmp") - invisible(saveAsTable(df, "parquetest", "parquet", mode="overwrite", path=parquetDataPath)) + parquetDataPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp") + invisible(saveAsTable(df, "parquetest", "parquet", mode = "overwrite", path = parquetDataPath)) df5 <- sql(hiveCtx, "select * from parquetest") expect_is(df5, "DataFrame") expect_equal(count(df5), 3) @@ -1094,7 +1094,7 @@ test_that("column binary mathfunctions", { "{\"a\":2, \"b\":6}", "{\"a\":3, \"b\":7}", "{\"a\":4, \"b\":8}") - jsonPathWithDup <- tempfile(pattern="sparkr-test", fileext=".tmp") + jsonPathWithDup <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(lines, jsonPathWithDup) df <- read.json(sqlContext, jsonPathWithDup) expect_equal(collect(select(df, atan2(df$a, df$b)))[1, "ATAN2(a, b)"], atan2(1, 5)) @@ -1244,7 +1244,7 @@ test_that("group by, agg functions", { df3 <- agg(gd, age = "stddev") expect_is(df3, "DataFrame") df3_local <- collect(df3) - expect_true(is.nan(df3_local[df3_local$name == "Andy",][1, 2])) + expect_true(is.nan(df3_local[df3_local$name == "Andy", ][1, 2])) df4 <- agg(gd, sumAge = sum(df$age)) expect_is(df4, "DataFrame") @@ -1264,34 +1264,34 @@ test_that("group by, agg functions", { "{\"name\":\"ID1\", \"value\": \"10\"}", "{\"name\":\"ID1\", \"value\": \"22\"}", "{\"name\":\"ID2\", \"value\": \"-3\"}") - jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") + jsonPath2 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(mockLines2, jsonPath2) gd2 <- groupBy(read.json(sqlContext, jsonPath2), "name") df6 <- agg(gd2, value = "sum") df6_local <- collect(df6) - expect_equal(42, df6_local[df6_local$name == "ID1",][1, 2]) - expect_equal(-3, df6_local[df6_local$name == "ID2",][1, 2]) + expect_equal(42, df6_local[df6_local$name == "ID1", ][1, 2]) + expect_equal(-3, df6_local[df6_local$name == "ID2", ][1, 2]) df7 <- agg(gd2, value = "stddev") df7_local <- collect(df7) - expect_true(abs(df7_local[df7_local$name == "ID1",][1, 2] - 6.928203) < 1e-6) - expect_true(is.nan(df7_local[df7_local$name == "ID2",][1, 2])) + expect_true(abs(df7_local[df7_local$name == "ID1", ][1, 2] - 6.928203) < 1e-6) + expect_true(is.nan(df7_local[df7_local$name == "ID2", ][1, 2])) mockLines3 <- c("{\"name\":\"Andy\", \"age\":30}", "{\"name\":\"Andy\", \"age\":30}", "{\"name\":\"Justin\", \"age\":19}", "{\"name\":\"Justin\", \"age\":1}") - jsonPath3 <- tempfile(pattern="sparkr-test", fileext=".tmp") + jsonPath3 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(mockLines3, jsonPath3) df8 <- read.json(sqlContext, jsonPath3) gd3 <- groupBy(df8, "name") gd3_local <- collect(sum(gd3)) - expect_equal(60, gd3_local[gd3_local$name == "Andy",][1, 2]) - expect_equal(20, gd3_local[gd3_local$name == "Justin",][1, 2]) + expect_equal(60, gd3_local[gd3_local$name == "Andy", ][1, 2]) + expect_equal(20, gd3_local[gd3_local$name == "Justin", ][1, 2]) expect_true(abs(collect(agg(df, sd(df$age)))[1, 1] - 7.778175) < 1e-6) gd3_local <- collect(agg(gd3, var(df8$age))) - expect_equal(162, gd3_local[gd3_local$name == "Justin",][1, 2]) + expect_equal(162, gd3_local[gd3_local$name == "Justin", ][1, 2]) # Test stats::sd, stats::var are working expect_true(abs(sd(1:2) - 0.7071068) < 1e-6) @@ -1304,10 +1304,10 @@ test_that("group by, agg functions", { test_that("arrange() and orderBy() on a DataFrame", { df <- read.json(sqlContext, jsonPath) sorted <- arrange(df, df$age) - expect_equal(collect(sorted)[1,2], "Michael") + expect_equal(collect(sorted)[1, 2], "Michael") sorted2 <- arrange(df, "name", decreasing = FALSE) - expect_equal(collect(sorted2)[2,"age"], 19) + expect_equal(collect(sorted2)[2, "age"], 19) sorted3 <- orderBy(df, asc(df$age)) expect_true(is.na(first(sorted3)$age)) @@ -1315,16 +1315,16 @@ test_that("arrange() and orderBy() on a DataFrame", { sorted4 <- orderBy(df, desc(df$name)) expect_equal(first(sorted4)$name, "Michael") - expect_equal(collect(sorted4)[3,"name"], "Andy") + expect_equal(collect(sorted4)[3, "name"], "Andy") sorted5 <- arrange(df, "age", "name", decreasing = TRUE) - expect_equal(collect(sorted5)[1,2], "Andy") + expect_equal(collect(sorted5)[1, 2], "Andy") - sorted6 <- arrange(df, "age","name", decreasing = c(T, F)) - expect_equal(collect(sorted6)[1,2], "Andy") + sorted6 <- arrange(df, "age", "name", decreasing = c(T, F)) + expect_equal(collect(sorted6)[1, 2], "Andy") sorted7 <- arrange(df, "name", decreasing = FALSE) - expect_equal(collect(sorted7)[2,"age"], 19) + expect_equal(collect(sorted7)[2, "age"], 19) }) test_that("filter() on a DataFrame", { @@ -1357,7 +1357,7 @@ test_that("join() and merge() on a DataFrame", { "{\"name\":\"Andy\", \"test\": \"no\"}", "{\"name\":\"Justin\", \"test\": \"yes\"}", "{\"name\":\"Bob\", \"test\": \"yes\"}") - jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") + jsonPath2 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(mockLines2, jsonPath2) df2 <- read.json(sqlContext, jsonPath2) @@ -1409,12 +1409,12 @@ test_that("join() and merge() on a DataFrame", { expect_equal(names(merged), c("age", "name_x", "name_y", "test")) expect_equal(collect(orderBy(merged, merged$name_x))$age[3], 19) - merged <- merge(df, df2, suffixes = c("-X","-Y")) + merged <- merge(df, df2, suffixes = c("-X", "-Y")) expect_equal(count(merged), 3) expect_equal(names(merged), c("age", "name-X", "name-Y", "test")) expect_equal(collect(orderBy(merged, merged$"name-X"))$age[1], 30) - merged <- merge(df, df2, by = "name", suffixes = c("-X","-Y"), sort = FALSE) + merged <- merge(df, df2, by = "name", suffixes = c("-X", "-Y"), sort = FALSE) expect_equal(count(merged), 3) expect_equal(names(merged), c("age", "name-X", "name-Y", "test")) expect_equal(collect(orderBy(merged, merged$"name-Y"))$"name-X"[3], "Michael") @@ -1432,7 +1432,7 @@ test_that("join() and merge() on a DataFrame", { "{\"name\":\"Andy\", \"name_y\":\"Andy\", \"test\": \"no\"}", "{\"name\":\"Justin\", \"name_y\":\"Justin\", \"test\": \"yes\"}", "{\"name\":\"Bob\", \"name_y\":\"Bob\", \"test\": \"yes\"}") - jsonPath3 <- tempfile(pattern="sparkr-test", fileext=".tmp") + jsonPath3 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(mockLines3, jsonPath3) df3 <- read.json(sqlContext, jsonPath3) expect_error(merge(df, df3), @@ -1460,8 +1460,8 @@ test_that("showDF()", { "|null|Michael|\n", "| 30| Andy|\n", "| 19| Justin|\n", - "+----+-------+\n", sep="") - expect_output(s , expected) + "+----+-------+\n", sep = "") + expect_output(s, expected) }) test_that("isLocal()", { @@ -1475,7 +1475,7 @@ test_that("unionAll(), rbind(), except(), and intersect() on a DataFrame", { lines <- c("{\"name\":\"Bob\", \"age\":24}", "{\"name\":\"Andy\", \"age\":30}", "{\"name\":\"James\", \"age\":35}") - jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") + jsonPath2 <- tempfile(pattern = "sparkr-test", fileext = ".tmp") writeLines(lines, jsonPath2) df2 <- read.df(sqlContext, jsonPath2, "json") @@ -1558,7 +1558,7 @@ test_that("mutate(), transform(), rename() and names()", { test_that("read/write Parquet files", { df <- read.df(sqlContext, jsonPath, "json") # Test write.df and read.df - write.df(df, parquetPath, "parquet", mode="overwrite") + write.df(df, parquetPath, "parquet", mode = "overwrite") df2 <- read.df(sqlContext, parquetPath, "parquet") expect_is(df2, "DataFrame") expect_equal(count(df2), 3) @@ -1593,7 +1593,7 @@ test_that("read/write text files", { expect_equal(colnames(df), c("value")) expect_equal(count(df), 3) textPath <- tempfile(pattern = "textPath", fileext = ".txt") - write.df(df, textPath, "text", mode="overwrite") + write.df(df, textPath, "text", mode = "overwrite") # Test write.text and read.text textPath2 <- tempfile(pattern = "textPath2", fileext = ".txt") @@ -1631,13 +1631,13 @@ test_that("dropna() and na.omit() on a DataFrame", { # drop with columns - expected <- rows[!is.na(rows$name),] + expected <- rows[!is.na(rows$name), ] actual <- collect(dropna(df, cols = "name")) expect_identical(expected, actual) actual <- collect(na.omit(df, cols = "name")) expect_identical(expected, actual) - expected <- rows[!is.na(rows$age),] + expected <- rows[!is.na(rows$age), ] actual <- collect(dropna(df, cols = "age")) row.names(expected) <- row.names(actual) # identical on two dataframes does not work here. Don't know why. @@ -1647,13 +1647,13 @@ test_that("dropna() and na.omit() on a DataFrame", { expect_identical(expected$name, actual$name) actual <- collect(na.omit(df, cols = "age")) - expected <- rows[!is.na(rows$age) & !is.na(rows$height),] + expected <- rows[!is.na(rows$age) & !is.na(rows$height), ] actual <- collect(dropna(df, cols = c("age", "height"))) expect_identical(expected, actual) actual <- collect(na.omit(df, cols = c("age", "height"))) expect_identical(expected, actual) - expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name),] + expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name), ] actual <- collect(dropna(df)) expect_identical(expected, actual) actual <- collect(na.omit(df)) @@ -1661,31 +1661,31 @@ test_that("dropna() and na.omit() on a DataFrame", { # drop with how - expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name),] + expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name), ] actual <- collect(dropna(df)) expect_identical(expected, actual) actual <- collect(na.omit(df)) expect_identical(expected, actual) - expected <- rows[!is.na(rows$age) | !is.na(rows$height) | !is.na(rows$name),] + expected <- rows[!is.na(rows$age) | !is.na(rows$height) | !is.na(rows$name), ] actual <- collect(dropna(df, "all")) expect_identical(expected, actual) actual <- collect(na.omit(df, "all")) expect_identical(expected, actual) - expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name),] + expected <- rows[!is.na(rows$age) & !is.na(rows$height) & !is.na(rows$name), ] actual <- collect(dropna(df, "any")) expect_identical(expected, actual) actual <- collect(na.omit(df, "any")) expect_identical(expected, actual) - expected <- rows[!is.na(rows$age) & !is.na(rows$height),] + expected <- rows[!is.na(rows$age) & !is.na(rows$height), ] actual <- collect(dropna(df, "any", cols = c("age", "height"))) expect_identical(expected, actual) actual <- collect(na.omit(df, "any", cols = c("age", "height"))) expect_identical(expected, actual) - expected <- rows[!is.na(rows$age) | !is.na(rows$height),] + expected <- rows[!is.na(rows$age) | !is.na(rows$height), ] actual <- collect(dropna(df, "all", cols = c("age", "height"))) expect_identical(expected, actual) actual <- collect(na.omit(df, "all", cols = c("age", "height"))) @@ -1693,7 +1693,7 @@ test_that("dropna() and na.omit() on a DataFrame", { # drop with threshold - expected <- rows[as.integer(!is.na(rows$age)) + as.integer(!is.na(rows$height)) >= 2,] + expected <- rows[as.integer(!is.na(rows$age)) + as.integer(!is.na(rows$height)) >= 2, ] actual <- collect(dropna(df, minNonNulls = 2, cols = c("age", "height"))) expect_identical(expected, actual) actual <- collect(na.omit(df, minNonNulls = 2, cols = c("age", "height"))) @@ -1701,7 +1701,7 @@ test_that("dropna() and na.omit() on a DataFrame", { expected <- rows[as.integer(!is.na(rows$age)) + as.integer(!is.na(rows$height)) + - as.integer(!is.na(rows$name)) >= 3,] + as.integer(!is.na(rows$name)) >= 3, ] actual <- collect(dropna(df, minNonNulls = 3, cols = c("name", "age", "height"))) expect_identical(expected, actual) actual <- collect(na.omit(df, minNonNulls = 3, cols = c("name", "age", "height"))) @@ -1754,7 +1754,7 @@ test_that("crosstab() on a DataFrame", { }) df <- toDF(rdd, list("a", "b")) ct <- crosstab(df, "a", "b") - ordered <- ct[order(ct$a_b),] + ordered <- ct[order(ct$a_b), ] row.names(ordered) <- NULL expected <- data.frame("a_b" = c("a0", "a1", "a2"), "b0" = c(1, 0, 1), "b1" = c(1, 1, 0), stringsAsFactors = FALSE, row.names = NULL) @@ -1782,10 +1782,10 @@ test_that("freqItems() on a DataFrame", { negDoubles = input * -1.0, stringsAsFactors = F) rdf[ input %% 3 == 0, ] <- c(1, "1", -1) df <- createDataFrame(sqlContext, rdf) - multiColResults <- freqItems(df, c("numbers", "letters"), support=0.1) + multiColResults <- freqItems(df, c("numbers", "letters"), support = 0.1) expect_true(1 %in% multiColResults$numbers[[1]]) expect_true("1" %in% multiColResults$letters[[1]]) - singleColResult <- freqItems(df, "negDoubles", support=0.1) + singleColResult <- freqItems(df, "negDoubles", support = 0.1) expect_true(-1 %in% head(singleColResult$negDoubles)[[1]]) l <- lapply(c(0:99), function(i) { @@ -1860,9 +1860,9 @@ test_that("with() on a DataFrame", { test_that("Method coltypes() to get and set R's data types of a DataFrame", { expect_equal(coltypes(irisDF), c(rep("numeric", 4), "character")) - data <- data.frame(c1=c(1,2,3), - c2=c(T,F,T), - c3=c("2015/01/01 10:00:00", "2015/01/02 10:00:00", "2015/01/03 10:00:00")) + data <- data.frame(c1 = c(1, 2, 3), + c2 = c(T, F, T), + c3 = c("2015/01/01 10:00:00", "2015/01/02 10:00:00", "2015/01/03 10:00:00")) schema <- structType(structField("c1", "byte"), structField("c3", "boolean"), @@ -1874,7 +1874,7 @@ test_that("Method coltypes() to get and set R's data types of a DataFrame", { # Test complex types x <- createDataFrame(sqlContext, list(list(as.environment( - list("a"="b", "c"="d", "e"="f"))))) + list("a" = "b", "c" = "d", "e" = "f"))))) expect_equal(coltypes(x), "map") df <- selectExpr(read.json(sqlContext, jsonPath), "name", "(age * 1.21) as age") @@ -1918,7 +1918,7 @@ test_that("Method str()", { # the number of columns. Therefore, it will suffice to check for the # number of returned rows x <- runif(200, 1, 10) - df <- data.frame(t(as.matrix(data.frame(x,x,x,x,x,x,x,x,x)))) + df <- data.frame(t(as.matrix(data.frame(x, x, x, x, x, x, x, x, x)))) DF <- createDataFrame(sqlContext, df) out <- capture.output(str(DF)) expect_equal(length(out), 103) diff --git a/R/pkg/inst/tests/testthat/test_textFile.R b/R/pkg/inst/tests/testthat/test_textFile.R index a9cf83dbdbdb1..e64ef1bb31a3a 100644 --- a/R/pkg/inst/tests/testthat/test_textFile.R +++ b/R/pkg/inst/tests/testthat/test_textFile.R @@ -23,7 +23,7 @@ sc <- sparkR.init() mockFile <- c("Spark is pretty.", "Spark is awesome.") test_that("textFile() on a local file returns an RDD", { - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName) rdd <- textFile(sc, fileName) @@ -35,7 +35,7 @@ test_that("textFile() on a local file returns an RDD", { }) test_that("textFile() followed by a collect() returns the same content", { - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName) rdd <- textFile(sc, fileName) @@ -45,7 +45,7 @@ test_that("textFile() followed by a collect() returns the same content", { }) test_that("textFile() word count works as expected", { - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName) rdd <- textFile(sc, fileName) @@ -63,7 +63,7 @@ test_that("textFile() word count works as expected", { }) test_that("several transformations on RDD created by textFile()", { - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName) rdd <- textFile(sc, fileName) # RDD @@ -77,8 +77,8 @@ test_that("several transformations on RDD created by textFile()", { }) test_that("textFile() followed by a saveAsTextFile() returns the same content", { - fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") - fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName1 <- tempfile(pattern = "spark-test", fileext = ".tmp") + fileName2 <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName1) rdd <- textFile(sc, fileName1, 1L) @@ -91,7 +91,7 @@ test_that("textFile() followed by a saveAsTextFile() returns the same content", }) test_that("saveAsTextFile() on a parallelized list works as expected", { - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") l <- list(1, 2, 3) rdd <- parallelize(sc, l, 1L) saveAsTextFile(rdd, fileName) @@ -102,8 +102,8 @@ test_that("saveAsTextFile() on a parallelized list works as expected", { }) test_that("textFile() and saveAsTextFile() word count works as expected", { - fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") - fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName1 <- tempfile(pattern = "spark-test", fileext = ".tmp") + fileName2 <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName1) rdd <- textFile(sc, fileName1) @@ -127,8 +127,8 @@ test_that("textFile() and saveAsTextFile() word count works as expected", { }) test_that("textFile() on multiple paths", { - fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") - fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName1 <- tempfile(pattern = "spark-test", fileext = ".tmp") + fileName2 <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines("Spark is pretty.", fileName1) writeLines("Spark is awesome.", fileName2) @@ -140,7 +140,7 @@ test_that("textFile() on multiple paths", { }) test_that("Pipelined operations on RDDs created using textFile", { - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName) rdd <- textFile(sc, fileName) diff --git a/R/pkg/inst/tests/testthat/test_utils.R b/R/pkg/inst/tests/testthat/test_utils.R index 56f14a3bce61e..4218138f641d1 100644 --- a/R/pkg/inst/tests/testthat/test_utils.R +++ b/R/pkg/inst/tests/testthat/test_utils.R @@ -41,7 +41,7 @@ test_that("convertJListToRList() gives back (deserializes) the original JLists test_that("serializeToBytes on RDD", { # File content mockFile <- c("Spark is pretty.", "Spark is awesome.") - fileName <- tempfile(pattern="spark-test", fileext=".tmp") + fileName <- tempfile(pattern = "spark-test", fileext = ".tmp") writeLines(mockFile, fileName) text.rdd <- textFile(sc, fileName) @@ -86,8 +86,8 @@ test_that("cleanClosure on R functions", { f <- function(x) { defUse <- base::as.integer(x) + 1 # Test for access operators `::`. lapply(x, g) + 1 # Test for capturing function call "g"'s closure as a argument of lapply. - l$field[1,1] <- 3 # Test for access operators `$`. - res <- defUse + l$field[1,] # Test for def-use chain of "defUse", and "" symbol. + l$field[1, 1] <- 3 # Test for access operators `$`. + res <- defUse + l$field[1, ] # Test for def-use chain of "defUse", and "" symbol. f(res) # Test for recursive calls. } newF <- cleanClosure(f) @@ -132,7 +132,7 @@ test_that("cleanClosure on R functions", { expect_equal(actual, expected) # Test for broadcast variables. - a <- matrix(nrow=10, ncol=10, data=rnorm(100)) + a <- matrix(nrow = 10, ncol = 10, data = rnorm(100)) aBroadcast <- broadcast(sc, a) normMultiply <- function(x) { norm(aBroadcast$value) * x } newnormMultiply <- SparkR:::cleanClosure(normMultiply) diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index 3ae072beca11b..b6784dbae3203 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -55,7 +55,7 @@ serializer <- SparkR:::readString(inputCon) # Include packages as required packageNames <- unserialize(SparkR:::readRaw(inputCon)) for (pkg in packageNames) { - suppressPackageStartupMessages(library(as.character(pkg), character.only=TRUE)) + suppressPackageStartupMessages(library(as.character(pkg), character.only = TRUE)) } # read function dependencies From f3daa099bf60edbd6ebf997c00e46db1e09f6dda Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sun, 13 Mar 2016 18:44:02 -0700 Subject: [PATCH 14/21] [SQL] fix typo in DataSourceRegister ## What changes were proposed in this pull request? fix typo in DataSourceRegister ## How was this patch tested? found when going through latest code Author: Jacky Li Closes #11686 from jackylk/patch-12. --- .../main/scala/org/apache/spark/sql/sources/interfaces.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index e251b52f6c0f9..be2d98c46d3b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -57,7 +57,7 @@ trait DataSourceRegister { * overridden by children to provide a nice alias for the data source. For example: * * {{{ - * override def format(): String = "parquet" + * override def shortName(): String = "parquet" * }}} * * @since 1.5.0 From 473263f9598d1cf880f421aae1b51eb0b6e3cf79 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 13 Mar 2016 18:47:04 -0700 Subject: [PATCH 15/21] [SPARK-13834][BUILD] Update sbt and sbt plugins for 2.x. ## What changes were proposed in this pull request? For 2.0.0, we had better make **sbt** and **sbt plugins** up-to-date. This PR checks the status of each plugins and bumps the followings. * sbt: 0.13.9 --> 0.13.11 * sbteclipse-plugin: 2.2.0 --> 4.0.0 * sbt-dependency-graph: 0.7.4 --> 0.8.2 * sbt-mima-plugin: 0.1.6 --> 0.1.9 * sbt-revolver: 0.7.2 --> 0.8.0 All other plugins are up-to-date. (Note that `sbt-avro` seems to be change from 0.3.2 to 1.0.1, but it's not published in the repository.) During upgrade, this PR also updated the following MiMa error. Note that the related excluding filter is already registered correctly. It seems due to the change of MiMa exception result. ``` // SPARK-12896 Send only accumulator updates to driver, not TaskMetrics ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.Accumulable.this"), -ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.Accumulator.this"), +ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.Accumulator.this"), ``` ## How was this patch tested? Pass the Jenkins build. Author: Dongjoon Hyun Closes #11669 from dongjoon-hyun/update_mima. --- dev/mima | 2 +- project/MimaExcludes.scala | 2 +- project/SparkBuild.scala | 5 ++--- project/build.properties | 2 +- project/plugins.sbt | 8 ++++---- 5 files changed, 9 insertions(+), 10 deletions(-) diff --git a/dev/mima b/dev/mima index b7f8d62b7d26f..c8e2df6cfcd4f 100755 --- a/dev/mima +++ b/dev/mima @@ -40,7 +40,7 @@ SPARK_PROFILES="-Pyarn -Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -P generate_mima_ignore "$(build/sbt $SPARK_PROFILES "export assembly/fullClasspath" | tail -n1)" generate_mima_ignore "$(build/sbt $SPARK_PROFILES "export oldDeps/fullClasspath" | tail -n1)" -echo -e "q\n" | build/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" +echo -e "q\n" | build/sbt mimaReportBinaryIssues | grep -v -e "info.*Resolving" ret_val=$? if [ $ret_val != 0 ]; then diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 567a717b9d24b..c4c8d8870f8b9 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -192,7 +192,7 @@ object MimaExcludes { ) ++ Seq( // SPARK-12896 Send only accumulator updates to driver, not TaskMetrics ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.Accumulable.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.Accumulator.this"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.Accumulator.this"), ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.Accumulator.initialValue") ) ++ Seq( // SPARK-12692 Scala style: Fix the style violation (Space before "," or ":") diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index e74fb174725d3..d1c67eac82663 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -26,7 +26,6 @@ import sbt.Classpaths.publishTask import sbt.Keys._ import sbtunidoc.Plugin.UnidocKeys.unidocGenjavadocVersion import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys} -import net.virtualvoid.sbt.graph.Plugin.graphSettings import spray.revolver.RevolverPlugin._ @@ -144,7 +143,7 @@ object SparkBuild extends PomBuild { "org.spark-project" %% "genjavadoc-plugin" % unidocGenjavadocVersion.value cross CrossVersion.full), scalacOptions <+= target.map(t => "-P:genjavadoc:out=" + (t / "java"))) - lazy val sharedSettings = graphSettings ++ sparkGenjavadocSettings ++ Seq ( + lazy val sharedSettings = sparkGenjavadocSettings ++ Seq ( javaHome := sys.env.get("JAVA_HOME") .orElse(sys.props.get("java.home").map { p => new File(p).getParentFile().getAbsolutePath() }) .map(file), @@ -241,7 +240,7 @@ object SparkBuild extends PomBuild { /* Enable shared settings on all projects */ (allProjects ++ optionallyEnabledProjects ++ assemblyProjects ++ Seq(spark, tools)) .foreach(enable(sharedSettings ++ DependencyOverrides.settings ++ - ExcludedDependencies.settings ++ Revolver.settings)) + ExcludedDependencies.settings)) /* Enable tests settings for all projects except examples, assembly and tools */ (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings)) diff --git a/project/build.properties b/project/build.properties index 86ca8755820a4..1e38156e0b577 100644 --- a/project/build.properties +++ b/project/build.properties @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. # -sbt.version=0.13.9 +sbt.version=0.13.11 diff --git a/project/plugins.sbt b/project/plugins.sbt index 822a7c4a82d5e..eeca94a47ce79 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -1,14 +1,14 @@ addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.11.2") -addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0") +addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "4.0.0") addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.6.0") -addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.4") +addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.8.2") addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.8.0") -addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") +addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.9") addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1") @@ -16,7 +16,7 @@ addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.3") addSbtPlugin("com.cavorite" % "sbt-avro" % "0.3.2") -addSbtPlugin("io.spray" % "sbt-revolver" % "0.7.2") +addSbtPlugin("io.spray" % "sbt-revolver" % "0.8.0") libraryDependencies += "org.ow2.asm" % "asm" % "5.0.3" From 184085284185011d7cc6d054b54d2d38eaf1dd77 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 13 Mar 2016 21:03:49 -0700 Subject: [PATCH 16/21] [SPARK-13823][CORE][STREAMING][SQL] Always specify Charset in String <-> byte[] conversions (and remaining Coverity items) ## What changes were proposed in this pull request? - Fixes calls to `new String(byte[])` or `String.getBytes()` that rely on platform default encoding, to use UTF-8 - Same for `InputStreamReader` and `OutputStreamWriter` constructors - Standardizes on UTF-8 everywhere - Standardizes specifying the encoding with `StandardCharsets.UTF-8`, not the Guava constant or "UTF-8" (which means handling `UnuspportedEncodingException`) - (also addresses the other remaining Coverity scan issues, which are pretty trivial; these are separated into commit https://github.com/srowen/spark/commit/1deecd8d9ca986d8adb1a42d315890ce5349d29c ) ## How was this patch tested? Jenkins tests Author: Sean Owen Closes #11657 from srowen/SPARK-13823. --- .../network/client/StreamInterceptor.java | 3 +- .../spark/network/protocol/Encoders.java | 8 ++--- .../spark/network/sasl/SparkSaslServer.java | 10 +++--- .../apache/spark/network/util/JavaUtils.java | 6 ++-- .../shuffle/ExternalShuffleBlockResolver.java | 12 +++---- .../ExternalShuffleBlockResolverSuite.java | 36 +++++++++++-------- .../shuffle/ExternalShuffleCleanupSuite.java | 21 ++++++----- .../apache/spark/unsafe/types/UTF8String.java | 10 ++---- .../apache/spark/api/python/PythonRDD.scala | 8 ++--- .../api/python/PythonWorkerFactory.scala | 3 +- .../WriteInputFormatTestDataGenerator.scala | 4 +-- .../scala/org/apache/spark/api/r/SerDe.scala | 3 +- .../spark/deploy/FaultToleranceTest.scala | 4 ++- .../spark/deploy/SparkSubmitArguments.scala | 3 +- .../deploy/rest/RestSubmissionClient.scala | 4 +-- .../spark/deploy/worker/DriverRunner.scala | 4 +-- .../spark/deploy/worker/ExecutorRunner.scala | 4 +-- .../scheduler/EventLoggingListener.scala | 4 +-- .../serializer/GenericAvroSerializer.scala | 3 +- .../scala/org/apache/spark/util/Utils.scala | 5 +-- .../java/org/apache/spark/JavaAPISuite.java | 4 +-- .../sort/ShuffleInMemorySorterSuite.java | 3 +- .../sort/UnsafeExternalSorterSuite.java | 2 -- .../sort/UnsafeInMemorySorterSuite.java | 3 +- .../org/apache/spark/SparkContextSuite.scala | 17 ++++----- .../spark/api/python/PythonRDDSuite.scala | 11 +++--- .../spark/deploy/SparkSubmitSuite.scala | 6 ++-- .../history/FsHistoryProviderSuite.scala | 9 ++--- .../deploy/history/HistoryServerSuite.scala | 6 ++-- .../rest/StandaloneRestSubmitSuite.scala | 4 +-- .../NettyBlockTransferSecuritySuite.scala | 7 ++-- .../apache/spark/util/FileAppenderSuite.scala | 12 +++---- .../org/apache/spark/util/UtilsSuite.scala | 16 ++++----- docs/streaming-custom-receivers.md | 6 ++-- .../streaming/JavaCustomReceiver.java | 4 ++- .../examples/streaming/CustomReceiver.scala | 4 ++- .../streaming/flume/sink/SparkSinkSuite.scala | 4 ++- .../streaming/flume/FlumeTestUtils.scala | 4 +-- .../flume/PollingFlumeTestUtils.scala | 5 +-- .../spark/streaming/kafka/KafkaUtils.scala | 4 +-- .../streaming/kinesis/KinesisTestUtils.scala | 3 +- .../kinesis/KPLBasedKinesisTestUtils.scala | 3 +- .../streaming/mqtt/MQTTInputDStream.scala | 4 ++- .../spark/streaming/mqtt/MQTTTestUtils.scala | 4 +-- .../spark/graphx/GraphLoaderSuite.scala | 3 +- .../launcher/AbstractCommandBuilder.java | 5 +-- .../spark/launcher/OutputRedirector.java | 3 +- .../SparkSubmitCommandBuilderSuite.java | 6 +--- .../mllib/api/python/PythonMLLibAPI.scala | 3 +- .../libsvm/JavaLibSVMRelationSuite.java | 4 +-- .../source/libsvm/LibSVMRelationSuite.scala | 6 ++-- .../spark/mllib/util/MLUtilsSuite.scala | 8 ++--- .../spark/sql/catalyst/parser/ParseUtils.java | 4 ++- .../sql/catalyst/expressions/literals.scala | 3 +- .../spark/sql/catalyst/util/package.scala | 3 +- .../expressions/LiteralExpressionSuite.scala | 4 ++- .../expressions/MathFunctionsSuite.scala | 6 ++-- .../expressions/MiscFunctionsSuite.scala | 19 ++++++---- .../expressions/UnsafeRowConverterSuite.scala | 14 ++++---- .../codegen/GeneratedProjectionSuite.scala | 5 ++- .../vectorized/ColumnVectorUtils.java | 3 +- .../execution/vectorized/ColumnarBatch.java | 4 +++ .../datasources/csv/CSVOptions.scala | 4 +-- .../execution/datasources/csv/CSVParser.scala | 3 +- .../datasources/csv/DefaultSource.scala | 4 +-- .../streaming/FileStreamSource.scala | 4 +-- .../spark/sql/sources/JavaSaveLoadSuite.java | 2 -- .../spark/sql/DataFrameFunctionsSuite.scala | 6 ++-- .../org/apache/spark/sql/DataFrameSuite.scala | 5 +-- .../spark/sql/MathExpressionsSuite.scala | 6 ++-- .../CompressionSchemeBenchmark.scala | 3 +- .../vectorized/ColumnarBatchBenchmark.scala | 4 ++- .../vectorized/ColumnarBatchSuite.scala | 21 ++++++----- .../sql/streaming/FileStreamSourceSuite.scala | 6 ++-- .../apache/spark/sql/test/SQLTestData.scala | 12 ++++--- .../sql/hive/thriftserver/CliSuite.scala | 3 +- .../HiveThriftServer2Suites.scala | 4 +-- .../apache/spark/sql/hive/HiveContext.scala | 3 +- .../hive/execution/ScriptTransformation.scala | 3 +- .../hive/JavaMetastoreDataSourcesSuite.java | 2 -- .../spark/sql/hive/orc/OrcQuerySuite.scala | 3 +- .../dstream/SocketInputDStream.scala | 4 ++- .../apache/spark/streaming/JavaAPISuite.java | 7 ++-- .../spark/streaming/JavaReceiverAPISuite.java | 4 ++- .../spark/streaming/CheckpointSuite.scala | 4 +-- .../spark/streaming/InputStreamsSuite.scala | 10 +++--- .../spark/streaming/MasterFailureTest.scala | 4 +-- .../org/apache/spark/deploy/yarn/Client.scala | 4 +-- .../deploy/yarn/BaseYarnClusterSuite.scala | 8 ++--- .../spark/deploy/yarn/YarnClusterSuite.scala | 12 +++---- .../yarn/YarnShuffleIntegrationSuite.scala | 4 +-- .../yarn/YarnSparkHadoopUtilSuite.scala | 3 +- 92 files changed, 321 insertions(+), 244 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/StreamInterceptor.java b/common/network-common/src/main/java/org/apache/spark/network/client/StreamInterceptor.java index 88ba3ccebdf20..b0e85bae7c309 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/StreamInterceptor.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/StreamInterceptor.java @@ -34,8 +34,7 @@ class StreamInterceptor implements TransportFrameDecoder.Interceptor { private final String streamId; private final long byteCount; private final StreamCallback callback; - - private volatile long bytesRead; + private long bytesRead; StreamInterceptor( TransportResponseHandler handler, diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java index 9162d0b977f83..be217522367c5 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java @@ -17,8 +17,8 @@ package org.apache.spark.network.protocol; +import java.nio.charset.StandardCharsets; -import com.google.common.base.Charsets; import io.netty.buffer.ByteBuf; /** Provides a canonical set of Encoders for simple types. */ @@ -27,11 +27,11 @@ public class Encoders { /** Strings are encoded with their length followed by UTF-8 bytes. */ public static class Strings { public static int encodedLength(String s) { - return 4 + s.getBytes(Charsets.UTF_8).length; + return 4 + s.getBytes(StandardCharsets.UTF_8).length; } public static void encode(ByteBuf buf, String s) { - byte[] bytes = s.getBytes(Charsets.UTF_8); + byte[] bytes = s.getBytes(StandardCharsets.UTF_8); buf.writeInt(bytes.length); buf.writeBytes(bytes); } @@ -40,7 +40,7 @@ public static String decode(ByteBuf buf) { int length = buf.readInt(); byte[] bytes = new byte[length]; buf.readBytes(bytes); - return new String(bytes, Charsets.UTF_8); + return new String(bytes, StandardCharsets.UTF_8); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java index 431cb67a2ae0b..b802a5af63c94 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java @@ -28,9 +28,9 @@ import javax.security.sasl.SaslException; import javax.security.sasl.SaslServer; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Map; -import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -187,14 +187,14 @@ public void handle(Callback[] callbacks) throws IOException, UnsupportedCallback /* Encode a byte[] identifier as a Base64-encoded string. */ public static String encodeIdentifier(String identifier) { Preconditions.checkNotNull(identifier, "User cannot be null if SASL is enabled"); - return Base64.encode(Unpooled.wrappedBuffer(identifier.getBytes(Charsets.UTF_8))) - .toString(Charsets.UTF_8); + return Base64.encode(Unpooled.wrappedBuffer(identifier.getBytes(StandardCharsets.UTF_8))) + .toString(StandardCharsets.UTF_8); } /** Encode a password as a base64-encoded char[] array. */ public static char[] encodePassword(String password) { Preconditions.checkNotNull(password, "Password cannot be null if SASL is enabled"); - return Base64.encode(Unpooled.wrappedBuffer(password.getBytes(Charsets.UTF_8))) - .toString(Charsets.UTF_8).toCharArray(); + return Base64.encode(Unpooled.wrappedBuffer(password.getBytes(StandardCharsets.UTF_8))) + .toString(StandardCharsets.UTF_8).toCharArray(); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java index ccc527306d920..8d83ae0712f0e 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -21,11 +21,11 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; -import com.google.common.base.Charsets; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import io.netty.buffer.Unpooled; @@ -68,7 +68,7 @@ public static int nonNegativeHash(Object obj) { * converted back to the same string through {@link #bytesToString(ByteBuffer)}. */ public static ByteBuffer stringToBytes(String s) { - return Unpooled.wrappedBuffer(s.getBytes(Charsets.UTF_8)).nioBuffer(); + return Unpooled.wrappedBuffer(s.getBytes(StandardCharsets.UTF_8)).nioBuffer(); } /** @@ -76,7 +76,7 @@ public static ByteBuffer stringToBytes(String s) { * converted back to the same byte buffer through {@link #stringToBytes(String)}. */ public static String bytesToString(ByteBuffer b) { - return Unpooled.wrappedBuffer(b).toString(Charsets.UTF_8); + return Unpooled.wrappedBuffer(b).toString(StandardCharsets.UTF_8); } /* diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index fe933ed650caf..460110d78f15b 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -18,6 +18,7 @@ package org.apache.spark.network.shuffle; import java.io.*; +import java.nio.charset.StandardCharsets; import java.util.*; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; @@ -27,7 +28,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Charsets; import com.google.common.base.Objects; import com.google.common.collect.Maps; import org.fusesource.leveldbjni.JniDBFactory; @@ -152,7 +152,7 @@ public void registerExecutor( try { if (db != null) { byte[] key = dbAppExecKey(fullId); - byte[] value = mapper.writeValueAsString(executorInfo).getBytes(Charsets.UTF_8); + byte[] value = mapper.writeValueAsString(executorInfo).getBytes(StandardCharsets.UTF_8); db.put(key, value); } } catch (Exception e) { @@ -350,7 +350,7 @@ private static byte[] dbAppExecKey(AppExecId appExecId) throws IOException { // we stick a common prefix on all the keys so we can find them in the DB String appExecJson = mapper.writeValueAsString(appExecId); String key = (APP_KEY_PREFIX + ";" + appExecJson); - return key.getBytes(Charsets.UTF_8); + return key.getBytes(StandardCharsets.UTF_8); } private static AppExecId parseDbAppExecKey(String s) throws IOException { @@ -368,10 +368,10 @@ static ConcurrentMap reloadRegisteredExecutors(D ConcurrentMap registeredExecutors = Maps.newConcurrentMap(); if (db != null) { DBIterator itr = db.iterator(); - itr.seek(APP_KEY_PREFIX.getBytes(Charsets.UTF_8)); + itr.seek(APP_KEY_PREFIX.getBytes(StandardCharsets.UTF_8)); while (itr.hasNext()) { Map.Entry e = itr.next(); - String key = new String(e.getKey(), Charsets.UTF_8); + String key = new String(e.getKey(), StandardCharsets.UTF_8); if (!key.startsWith(APP_KEY_PREFIX)) { break; } @@ -418,7 +418,7 @@ private static void storeVersion(DB db) throws IOException { public static class StoreVersion { - static final byte[] KEY = "StoreVersion".getBytes(Charsets.UTF_8); + static final byte[] KEY = "StoreVersion".getBytes(StandardCharsets.UTF_8); public final int major; public final int minor; diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index 60a1b8b0451fe..d9b5f0261aaba 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.io.CharStreams; @@ -34,15 +35,16 @@ import static org.junit.Assert.*; public class ExternalShuffleBlockResolverSuite { - static String sortBlock0 = "Hello!"; - static String sortBlock1 = "World!"; + private static final String sortBlock0 = "Hello!"; + private static final String sortBlock1 = "World!"; - static String hashBlock0 = "Elementary"; - static String hashBlock1 = "Tabular"; + private static final String hashBlock0 = "Elementary"; + private static final String hashBlock1 = "Tabular"; - static TestShuffleDataContext dataContext; + private static TestShuffleDataContext dataContext; - static TransportConf conf = new TransportConf("shuffle", new SystemPropertyConfigProvider()); + private static final TransportConf conf = + new TransportConf("shuffle", new SystemPropertyConfigProvider()); @BeforeClass public static void beforeAll() throws IOException { @@ -50,10 +52,12 @@ public static void beforeAll() throws IOException { dataContext.create(); // Write some sort and hash data. - dataContext.insertSortShuffleData(0, 0, - new byte[][] { sortBlock0.getBytes(), sortBlock1.getBytes() } ); - dataContext.insertHashShuffleData(1, 0, - new byte[][] { hashBlock0.getBytes(), hashBlock1.getBytes() } ); + dataContext.insertSortShuffleData(0, 0, new byte[][] { + sortBlock0.getBytes(StandardCharsets.UTF_8), + sortBlock1.getBytes(StandardCharsets.UTF_8)}); + dataContext.insertHashShuffleData(1, 0, new byte[][] { + hashBlock0.getBytes(StandardCharsets.UTF_8), + hashBlock1.getBytes(StandardCharsets.UTF_8)}); } @AfterClass @@ -100,13 +104,15 @@ public void testSortShuffleBlocks() throws IOException { InputStream block0Stream = resolver.getBlockData("app0", "exec0", "shuffle_0_0_0").createInputStream(); - String block0 = CharStreams.toString(new InputStreamReader(block0Stream)); + String block0 = CharStreams.toString( + new InputStreamReader(block0Stream, StandardCharsets.UTF_8)); block0Stream.close(); assertEquals(sortBlock0, block0); InputStream block1Stream = resolver.getBlockData("app0", "exec0", "shuffle_0_0_1").createInputStream(); - String block1 = CharStreams.toString(new InputStreamReader(block1Stream)); + String block1 = CharStreams.toString( + new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); block1Stream.close(); assertEquals(sortBlock1, block1); } @@ -119,13 +125,15 @@ public void testHashShuffleBlocks() throws IOException { InputStream block0Stream = resolver.getBlockData("app0", "exec0", "shuffle_1_0_0").createInputStream(); - String block0 = CharStreams.toString(new InputStreamReader(block0Stream)); + String block0 = CharStreams.toString( + new InputStreamReader(block0Stream, StandardCharsets.UTF_8)); block0Stream.close(); assertEquals(hashBlock0, block0); InputStream block1Stream = resolver.getBlockData("app0", "exec0", "shuffle_1_0_1").createInputStream(); - String block1 = CharStreams.toString(new InputStreamReader(block1Stream)); + String block1 = CharStreams.toString( + new InputStreamReader(block1Stream, StandardCharsets.UTF_8)); block1Stream.close(); assertEquals(hashBlock1, block1); } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java index 532d7ab8d01bd..43d0201405872 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java @@ -19,6 +19,7 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Random; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; @@ -34,8 +35,8 @@ public class ExternalShuffleCleanupSuite { // Same-thread Executor used to ensure cleanup happens synchronously in test thread. - Executor sameThreadExecutor = MoreExecutors.sameThreadExecutor(); - TransportConf conf = new TransportConf("shuffle", new SystemPropertyConfigProvider()); + private Executor sameThreadExecutor = MoreExecutors.sameThreadExecutor(); + private TransportConf conf = new TransportConf("shuffle", new SystemPropertyConfigProvider()); @Test public void noCleanupAndCleanup() throws IOException { @@ -123,27 +124,29 @@ public void cleanupOnlyRemovedApp() throws IOException { assertCleanedUp(dataContext1); } - private void assertStillThere(TestShuffleDataContext dataContext) { + private static void assertStillThere(TestShuffleDataContext dataContext) { for (String localDir : dataContext.localDirs) { assertTrue(localDir + " was cleaned up prematurely", new File(localDir).exists()); } } - private void assertCleanedUp(TestShuffleDataContext dataContext) { + private static void assertCleanedUp(TestShuffleDataContext dataContext) { for (String localDir : dataContext.localDirs) { assertFalse(localDir + " wasn't cleaned up", new File(localDir).exists()); } } - private TestShuffleDataContext createSomeData() throws IOException { + private static TestShuffleDataContext createSomeData() throws IOException { Random rand = new Random(123); TestShuffleDataContext dataContext = new TestShuffleDataContext(10, 5); dataContext.create(); - dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), - new byte[][] { "ABC".getBytes(), "DEF".getBytes() } ); - dataContext.insertHashShuffleData(rand.nextInt(1000), rand.nextInt(1000) + 1000, - new byte[][] { "GHI".getBytes(), "JKLMNOPQRSTUVWXYZ".getBytes() } ); + dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), new byte[][] { + "ABC".getBytes(StandardCharsets.UTF_8), + "DEF".getBytes(StandardCharsets.UTF_8)}); + dataContext.insertHashShuffleData(rand.nextInt(1000), rand.nextInt(1000) + 1000, new byte[][] { + "GHI".getBytes(StandardCharsets.UTF_8), + "JKLMNOPQRSTUVWXYZ".getBytes(StandardCharsets.UTF_8)}); return dataContext; } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index 427a8315e02b7..e16166ade4e5d 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -21,6 +21,7 @@ import java.io.*; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Map; @@ -825,14 +826,7 @@ public UTF8String translate(Map dict) { @Override public String toString() { - try { - return new String(getBytes(), "utf-8"); - } catch (UnsupportedEncodingException e) { - // Turn the exception into unchecked so we can find out about it at runtime, but - // don't need to add lots of boilerplate code everywhere. - throwException(e); - return "unknown"; // we will never reach here. - } + return new String(getBytes(), StandardCharsets.UTF_8); } @Override diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 05d1c31a08f22..8f306770a184f 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -19,6 +19,7 @@ package org.apache.spark.api.python import java.io._ import java.net._ +import java.nio.charset.StandardCharsets import java.util.{ArrayList => JArrayList, Collections, List => JList, Map => JMap} import scala.collection.JavaConverters._ @@ -26,7 +27,6 @@ import scala.collection.mutable import scala.language.existentials import scala.util.control.NonFatal -import com.google.common.base.Charsets.UTF_8 import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{InputFormat, JobConf, OutputFormat} @@ -165,7 +165,7 @@ private[spark] class PythonRunner( val exLength = stream.readInt() val obj = new Array[Byte](exLength) stream.readFully(obj) - throw new PythonException(new String(obj, UTF_8), + throw new PythonException(new String(obj, StandardCharsets.UTF_8), writerThread.exception.getOrElse(null)) case SpecialLengths.END_OF_DATA_SECTION => // We've finished the data section of the output, but we can still @@ -624,7 +624,7 @@ private[spark] object PythonRDD extends Logging { } def writeUTF(str: String, dataOut: DataOutputStream) { - val bytes = str.getBytes(UTF_8) + val bytes = str.getBytes(StandardCharsets.UTF_8) dataOut.writeInt(bytes.length) dataOut.write(bytes) } @@ -817,7 +817,7 @@ private[spark] object PythonRDD extends Logging { private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] { - override def call(arr: Array[Byte]) : String = new String(arr, UTF_8) + override def call(arr: Array[Byte]) : String = new String(arr, StandardCharsets.UTF_8) } /** diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index a2a2f89f1e875..433764be89fb7 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -19,6 +19,7 @@ package org.apache.spark.api.python import java.io.{DataInputStream, DataOutputStream, InputStream, OutputStreamWriter} import java.net.{InetAddress, ServerSocket, Socket, SocketException} +import java.nio.charset.StandardCharsets import java.util.Arrays import scala.collection.mutable @@ -121,7 +122,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String redirectStreamsToStderr(worker.getInputStream, worker.getErrorStream) // Tell the worker our port - val out = new OutputStreamWriter(worker.getOutputStream) + val out = new OutputStreamWriter(worker.getOutputStream, StandardCharsets.UTF_8) out.write(serverSocket.getLocalPort + "\n") out.flush() diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala index 9549784aeabf5..34cb7c61d7034 100644 --- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala +++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala @@ -19,10 +19,10 @@ package org.apache.spark.api.python import java.{util => ju} import java.io.{DataInput, DataOutput} +import java.nio.charset.StandardCharsets import scala.collection.JavaConverters._ -import com.google.common.base.Charsets.UTF_8 import org.apache.hadoop.io._ import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat @@ -134,7 +134,7 @@ object WriteInputFormatTestDataGenerator { sc.parallelize(intKeys).saveAsSequenceFile(intPath) sc.parallelize(intKeys.map{ case (k, v) => (k.toDouble, v) }).saveAsSequenceFile(doublePath) sc.parallelize(intKeys.map{ case (k, v) => (k.toString, v) }).saveAsSequenceFile(textPath) - sc.parallelize(intKeys.map{ case (k, v) => (k, v.getBytes(UTF_8)) } + sc.parallelize(intKeys.map{ case (k, v) => (k, v.getBytes(StandardCharsets.UTF_8)) } ).saveAsSequenceFile(bytesPath) val bools = Seq((1, true), (2, true), (2, false), (3, true), (2, false), (1, false)) sc.parallelize(bools).saveAsSequenceFile(boolPath) diff --git a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala index af815f885e8ae..c7fb192f26bd0 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -18,6 +18,7 @@ package org.apache.spark.api.r import java.io.{DataInputStream, DataOutputStream} +import java.nio.charset.StandardCharsets import java.sql.{Date, Time, Timestamp} import scala.collection.JavaConverters._ @@ -109,7 +110,7 @@ private[spark] object SerDe { val bytes = new Array[Byte](len) in.readFully(bytes) assert(bytes(len - 1) == 0) - val str = new String(bytes.dropRight(1), "UTF-8") + val str = new String(bytes.dropRight(1), StandardCharsets.UTF_8) str } diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 434aadd2c61ac..305994a3f3543 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy import java.io._ import java.net.URL +import java.nio.charset.StandardCharsets import java.util.concurrent.TimeoutException import scala.collection.mutable.ListBuffer @@ -348,7 +349,8 @@ private class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile def readState() { try { - val masterStream = new InputStreamReader(new URL("http://%s:8080/json".format(ip)).openStream) + val masterStream = new InputStreamReader( + new URL("http://%s:8080/json".format(ip)).openStream, StandardCharsets.UTF_8) val json = JsonMethods.parse(masterStream) val workers = json \ "workers" diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 175756b80b6bb..a62096d771724 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy import java.io.{ByteArrayOutputStream, PrintStream} import java.lang.reflect.InvocationTargetException import java.net.URI +import java.nio.charset.StandardCharsets import java.util.{List => JList} import java.util.jar.JarFile @@ -608,7 +609,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S stream.flush() // Get the output and discard any unnecessary lines from it. - Source.fromString(new String(out.toByteArray())).getLines + Source.fromString(new String(out.toByteArray(), StandardCharsets.UTF_8)).getLines .filter { line => !line.startsWith("log4j") && !line.startsWith("usage") } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index 006e2e1472580..d3e092a34c172 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.rest import java.io.{DataOutputStream, FileNotFoundException} import java.net.{ConnectException, HttpURLConnection, SocketException, URL} +import java.nio.charset.StandardCharsets import java.util.concurrent.TimeoutException import javax.servlet.http.HttpServletResponse @@ -28,7 +29,6 @@ import scala.concurrent.duration._ import scala.io.Source import com.fasterxml.jackson.core.JsonProcessingException -import com.google.common.base.Charsets import org.apache.spark.{Logging, SPARK_VERSION => sparkVersion, SparkConf} import org.apache.spark.util.Utils @@ -211,7 +211,7 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { try { val out = new DataOutputStream(conn.getOutputStream) Utils.tryWithSafeFinally { - out.write(json.getBytes(Charsets.UTF_8)) + out.write(json.getBytes(StandardCharsets.UTF_8)) } { out.close() } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 6049db6d989ae..7f4fe26c0d15e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -18,10 +18,10 @@ package org.apache.spark.deploy.worker import java.io._ +import java.nio.charset.StandardCharsets import scala.collection.JavaConverters._ -import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.hadoop.fs.Path @@ -174,7 +174,7 @@ private[deploy] class DriverRunner( val stderr = new File(baseDir, "stderr") val formattedCommand = builder.command.asScala.mkString("\"", "\" \"", "\"") val header = "Launch Command: %s\n%s\n\n".format(formattedCommand, "=" * 40) - Files.append(header, stderr, UTF_8) + Files.append(header, stderr, StandardCharsets.UTF_8) CommandUtils.redirectStream(process.getErrorStream, stderr) } runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index c6687a4c63a6a..208a1bb68edb9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -18,10 +18,10 @@ package org.apache.spark.deploy.worker import java.io._ +import java.nio.charset.StandardCharsets import scala.collection.JavaConverters._ -import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.spark.{Logging, SecurityManager, SparkConf} @@ -168,7 +168,7 @@ private[deploy] class ExecutorRunner( stdoutAppender = FileAppender(process.getInputStream, stdout, conf) val stderr = new File(executorDir, "stderr") - Files.write(header, stderr, UTF_8) + Files.write(header, stderr, StandardCharsets.UTF_8) stderrAppender = FileAppender(process.getErrorStream, stderr, conf) // Wait for it to exit; executor may exit with code 0 (when driver instructs it to shutdown) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 8354e2a6112a2..2d76d08af6cdd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -19,11 +19,11 @@ package org.apache.spark.scheduler import java.io._ import java.net.URI +import java.nio.charset.StandardCharsets import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import com.google.common.base.Charsets import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} import org.apache.hadoop.fs.permission.FsPermission @@ -254,7 +254,7 @@ private[spark] object EventLoggingListener extends Logging { def initEventLog(logStream: OutputStream): Unit = { val metadata = SparkListenerLogStart(SPARK_VERSION) val metadataJson = compact(JsonProtocol.logStartToJson(metadata)) + "\n" - logStream.write(metadataJson.getBytes(Charsets.UTF_8)) + logStream.write(metadataJson.getBytes(StandardCharsets.UTF_8)) } /** diff --git a/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala index 3d5b7105f0ca8..1a8e545b4f59e 100644 --- a/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala @@ -19,6 +19,7 @@ package org.apache.spark.serializer import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets import scala.collection.mutable @@ -86,7 +87,7 @@ private[serializer] class GenericAvroSerializer(schemas: Map[Long, String]) schemaBytes.arrayOffset() + schemaBytes.position(), schemaBytes.remaining()) val bytes = IOUtils.toByteArray(codec.compressedInputStream(bis)) - new Schema.Parser().parse(new String(bytes, "UTF-8")) + new Schema.Parser().parse(new String(bytes, StandardCharsets.UTF_8)) }) /** 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 b4c49513711c2..b5a98ce569a8a 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -22,6 +22,7 @@ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer import java.nio.channels.Channels +import java.nio.charset.StandardCharsets import java.nio.file.Files import java.util.{Locale, Properties, Random, UUID} import java.util.concurrent._ @@ -1904,7 +1905,7 @@ private[spark] object Utils extends Logging { require(file.exists(), s"Properties file $file does not exist") require(file.isFile(), s"Properties file $file is not a normal file") - val inReader = new InputStreamReader(new FileInputStream(file), "UTF-8") + val inReader = new InputStreamReader(new FileInputStream(file), StandardCharsets.UTF_8) try { val properties = new Properties() properties.load(inReader) @@ -2344,7 +2345,7 @@ private[spark] class CircularBuffer(sizeInBytes: Int = 10240) extends java.io.Ou def read(): Int = if (iterator.hasNext) iterator.next() else -1 } - val reader = new BufferedReader(new InputStreamReader(input)) + val reader = new BufferedReader(new InputStreamReader(input, StandardCharsets.UTF_8)) val stringBuilder = new StringBuilder var line = reader.readLine() while (line != null) { diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index e6a4ab7550c2a..a7e74c00793c3 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -21,6 +21,7 @@ import java.nio.channels.FileChannel; import java.nio.ByteBuffer; import java.net.URI; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -45,7 +46,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.base.Throwables; -import com.google.common.base.Charsets; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.Text; @@ -1058,7 +1058,7 @@ public void textFiles() throws IOException { rdd.saveAsTextFile(outputDir); // Read the plain text file and check it's OK File outputFile = new File(outputDir, "part-00000"); - String content = Files.toString(outputFile, Charsets.UTF_8); + String content = Files.toString(outputFile, StandardCharsets.UTF_8); Assert.assertEquals("1\n2\n3\n4\n", content); // Also try reading it in as a text file RDD List expected = Arrays.asList("1", "2", "3", "4"); diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java index b4fa33f32a3fd..a3502708aadec 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java @@ -17,6 +17,7 @@ package org.apache.spark.shuffle.sort; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Random; @@ -41,7 +42,7 @@ public class ShuffleInMemorySorterSuite { private static String getStringFromDataPage(Object baseObject, long baseOffset, int strLength) { final byte[] strBytes = new byte[strLength]; Platform.copyMemory(baseObject, baseOffset, strBytes, Platform.BYTE_ARRAY_OFFSET, strLength); - return new String(strBytes); + return new String(strBytes, StandardCharsets.UTF_8); } @Test diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java index b757ddc3b37f9..a79ed58133f1b 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java @@ -80,7 +80,6 @@ public int compare( } }; - SparkConf sparkConf; File tempDir; @Mock(answer = RETURNS_SMART_NULLS) BlockManager blockManager; @Mock(answer = RETURNS_SMART_NULLS) DiskBlockManager diskBlockManager; @@ -99,7 +98,6 @@ public OutputStream apply(OutputStream stream) { @Before public void setUp() { MockitoAnnotations.initMocks(this); - sparkConf = new SparkConf(); tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "unsafe-test"); spillFilesCreated.clear(); taskContext = mock(TaskContext.class); diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java index ff41768df1d8f..90849ab0bd8f3 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java @@ -17,6 +17,7 @@ package org.apache.spark.util.collection.unsafe.sort; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import org.junit.Assert; @@ -41,7 +42,7 @@ public class UnsafeInMemorySorterSuite { private static String getStringFromDataPage(Object baseObject, long baseOffset, int length) { final byte[] strBytes = new byte[length]; Platform.copyMemory(baseObject, baseOffset, strBytes, Platform.BYTE_ARRAY_OFFSET, length); - return new String(strBytes); + return new String(strBytes, StandardCharsets.UTF_8); } @Test diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 556afd08bbfe5..841fd02ae8bb6 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -18,12 +18,12 @@ package org.apache.spark import java.io.File +import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit import scala.concurrent.Await import scala.concurrent.duration.Duration -import com.google.common.base.Charsets._ import com.google.common.io.Files import org.apache.hadoop.io.{BytesWritable, LongWritable, Text} import org.apache.hadoop.mapred.TextInputFormat @@ -115,8 +115,8 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { val absolutePath2 = file2.getAbsolutePath try { - Files.write("somewords1", file1, UTF_8) - Files.write("somewords2", file2, UTF_8) + Files.write("somewords1", file1, StandardCharsets.UTF_8) + Files.write("somewords2", file2, StandardCharsets.UTF_8) val length1 = file1.length() val length2 = file2.length() @@ -243,11 +243,12 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext { try { // Create 5 text files. - Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", file1, UTF_8) - Files.write("someline1 in file2\nsomeline2 in file2", file2, UTF_8) - Files.write("someline1 in file3", file3, UTF_8) - Files.write("someline1 in file4\nsomeline2 in file4", file4, UTF_8) - Files.write("someline1 in file2\nsomeline2 in file5", file5, UTF_8) + Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", file1, + StandardCharsets.UTF_8) + Files.write("someline1 in file2\nsomeline2 in file2", file2, StandardCharsets.UTF_8) + Files.write("someline1 in file3", file3, StandardCharsets.UTF_8) + Files.write("someline1 in file4\nsomeline2 in file4", file4, StandardCharsets.UTF_8) + Files.write("someline1 in file2\nsomeline2 in file5", file5, StandardCharsets.UTF_8) sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala index 41f2a5c972b6b..05b4e67412f2e 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.api.python import java.io.{ByteArrayOutputStream, DataOutputStream} +import java.nio.charset.StandardCharsets import org.apache.spark.SparkFunSuite @@ -35,10 +36,12 @@ class PythonRDDSuite extends SparkFunSuite { // The correctness will be tested in Python PythonRDD.writeIteratorToStream(Iterator("a", null), buffer) PythonRDD.writeIteratorToStream(Iterator(null, "a"), buffer) - PythonRDD.writeIteratorToStream(Iterator("a".getBytes, null), buffer) - PythonRDD.writeIteratorToStream(Iterator(null, "a".getBytes), buffer) + PythonRDD.writeIteratorToStream(Iterator("a".getBytes(StandardCharsets.UTF_8), null), buffer) + PythonRDD.writeIteratorToStream(Iterator(null, "a".getBytes(StandardCharsets.UTF_8)), buffer) PythonRDD.writeIteratorToStream(Iterator((null, null), ("a", null), (null, "b")), buffer) - PythonRDD.writeIteratorToStream( - Iterator((null, null), ("a".getBytes, null), (null, "b".getBytes)), buffer) + PythonRDD.writeIteratorToStream(Iterator( + (null, null), + ("a".getBytes(StandardCharsets.UTF_8), null), + (null, "b".getBytes(StandardCharsets.UTF_8))), buffer) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 41ac60ece0eda..91fef772d13c3 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -18,10 +18,10 @@ package org.apache.spark.deploy import java.io._ +import java.nio.charset.StandardCharsets import scala.collection.mutable.ArrayBuffer -import com.google.common.base.Charsets.UTF_8 import com.google.common.io.ByteStreams import org.scalatest.{BeforeAndAfterEach, Matchers} import org.scalatest.concurrent.Timeouts @@ -593,7 +593,7 @@ class SparkSubmitSuite val tmpDir = Utils.createTempDir() val defaultsConf = new File(tmpDir.getAbsolutePath, "spark-defaults.conf") - val writer = new OutputStreamWriter(new FileOutputStream(defaultsConf)) + val writer = new OutputStreamWriter(new FileOutputStream(defaultsConf), StandardCharsets.UTF_8) for ((key, value) <- defaults) writer.write(s"$key $value\n") writer.close() @@ -661,7 +661,7 @@ object UserClasspathFirstTest { val ccl = Thread.currentThread().getContextClassLoader() val resource = ccl.getResourceAsStream("test.resource") val bytes = ByteStreams.toByteArray(resource) - val contents = new String(bytes, 0, bytes.length, UTF_8) + val contents = new String(bytes, 0, bytes.length, StandardCharsets.UTF_8) if (contents != "USER") { throw new SparkException("Should have read user resource, but instead read: " + contents) } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 8e8007f4ebf4b..5fd599e190c7c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -20,13 +20,13 @@ package org.apache.spark.deploy.history import java.io.{BufferedOutputStream, ByteArrayInputStream, ByteArrayOutputStream, File, FileOutputStream, OutputStreamWriter} import java.net.URI +import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit import java.util.zip.{ZipInputStream, ZipOutputStream} import scala.concurrent.duration._ import scala.language.postfixOps -import com.google.common.base.Charsets import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.hdfs.DistributedFileSystem import org.json4s.jackson.JsonMethods._ @@ -320,8 +320,9 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc var entry = inputStream.getNextEntry entry should not be null while (entry != null) { - val actual = new String(ByteStreams.toByteArray(inputStream), Charsets.UTF_8) - val expected = Files.toString(logs.find(_.getName == entry.getName).get, Charsets.UTF_8) + val actual = new String(ByteStreams.toByteArray(inputStream), StandardCharsets.UTF_8) + val expected = + Files.toString(logs.find(_.getName == entry.getName).get, StandardCharsets.UTF_8) actual should be (expected) totalEntries += 1 entry = inputStream.getNextEntry @@ -415,7 +416,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc if (isNewFormat) { EventLoggingListener.initEventLog(new FileOutputStream(file)) } - val writer = new OutputStreamWriter(bstream, "UTF-8") + val writer = new OutputStreamWriter(bstream, StandardCharsets.UTF_8) Utils.tryWithSafeFinally { events.foreach(e => writer.write(compact(render(JsonProtocol.sparkEventToJson(e))) + "\n")) } { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index e5cd2eddba1e8..5822261d8da75 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.history import java.io.{File, FileInputStream, FileWriter, InputStream, IOException} import java.net.{HttpURLConnection, URL} +import java.nio.charset.StandardCharsets import java.util.zip.ZipInputStream import javax.servlet.http.{HttpServletRequest, HttpServletResponse} @@ -25,7 +26,6 @@ import scala.concurrent.duration._ import scala.language.postfixOps import com.codahale.metrics.Counter -import com.google.common.base.Charsets import com.google.common.io.{ByteStreams, Files} import org.apache.commons.io.{FileUtils, IOUtils} import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} @@ -216,8 +216,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers val expectedFile = { new File(logDir, entry.getName) } - val expected = Files.toString(expectedFile, Charsets.UTF_8) - val actual = new String(ByteStreams.toByteArray(zipStream), Charsets.UTF_8) + val expected = Files.toString(expectedFile, StandardCharsets.UTF_8) + val actual = new String(ByteStreams.toByteArray(zipStream), StandardCharsets.UTF_8) actual should be (expected) filesCompared += 1 } diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index ee889bf144546..a7bb9aa4686eb 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -19,11 +19,11 @@ package org.apache.spark.deploy.rest import java.io.DataOutputStream import java.net.{HttpURLConnection, URL} +import java.nio.charset.StandardCharsets import javax.servlet.http.HttpServletResponse import scala.collection.mutable -import com.google.common.base.Charsets import org.json4s.JsonAST._ import org.json4s.jackson.JsonMethods._ import org.scalatest.BeforeAndAfterEach @@ -498,7 +498,7 @@ class StandaloneRestSubmitSuite extends SparkFunSuite with BeforeAndAfterEach { if (body.nonEmpty) { conn.setDoOutput(true) val out = new DataOutputStream(conn.getOutputStream) - out.write(body.getBytes(Charsets.UTF_8)) + out.write(body.getBytes(StandardCharsets.UTF_8)) out.close() } conn diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index 47dbcb8fc0eaa..02806a16b9467 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.network.netty import java.io.InputStreamReader import java.nio._ -import java.nio.charset.Charset +import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit import scala.concurrent.{Await, Promise} @@ -103,7 +103,8 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi val blockManager = mock[BlockDataManager] val blockId = ShuffleBlockId(0, 1, 2) val blockString = "Hello, world!" - val blockBuffer = new NioManagedBuffer(ByteBuffer.wrap(blockString.getBytes)) + val blockBuffer = new NioManagedBuffer(ByteBuffer.wrap( + blockString.getBytes(StandardCharsets.UTF_8))) when(blockManager.getBlockData(blockId)).thenReturn(blockBuffer) val securityManager0 = new SecurityManager(conf0) @@ -117,7 +118,7 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi val result = fetchBlock(exec0, exec1, "1", blockId) match { case Success(buf) => val actualString = CharStreams.toString( - new InputStreamReader(buf.createInputStream(), Charset.forName("UTF-8"))) + new InputStreamReader(buf.createInputStream(), StandardCharsets.UTF_8)) actualString should equal(blockString) buf.release() Success() diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index b367cc8358342..d30eafd2d4218 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -18,12 +18,12 @@ package org.apache.spark.util import java.io._ +import java.nio.charset.StandardCharsets import java.util.concurrent.CountDownLatch import scala.collection.mutable.HashSet import scala.reflect._ -import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.log4j.{Appender, Level, Logger} import org.apache.log4j.spi.LoggingEvent @@ -48,11 +48,11 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { test("basic file appender") { val testString = (1 to 1000).mkString(", ") - val inputStream = new ByteArrayInputStream(testString.getBytes(UTF_8)) + val inputStream = new ByteArrayInputStream(testString.getBytes(StandardCharsets.UTF_8)) val appender = new FileAppender(inputStream, testFile) inputStream.close() appender.awaitTermination() - assert(Files.toString(testFile, UTF_8) === testString) + assert(Files.toString(testFile, StandardCharsets.UTF_8) === testString) } test("rolling file appender - time-based rolling") { @@ -100,7 +100,7 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { val allGeneratedFiles = new HashSet[String]() val items = (1 to 10).map { _.toString * 10000 } for (i <- 0 until items.size) { - testOutputStream.write(items(i).getBytes(UTF_8)) + testOutputStream.write(items(i).getBytes(StandardCharsets.UTF_8)) testOutputStream.flush() allGeneratedFiles ++= RollingFileAppender.getSortedRolledOverFiles( testFile.getParentFile.toString, testFile.getName).map(_.toString) @@ -267,7 +267,7 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { // send data to appender through the input stream, and wait for the data to be written val expectedText = textToAppend.mkString("") for (i <- 0 until textToAppend.size) { - outputStream.write(textToAppend(i).getBytes(UTF_8)) + outputStream.write(textToAppend(i).getBytes(StandardCharsets.UTF_8)) outputStream.flush() Thread.sleep(sleepTimeBetweenTexts) } @@ -282,7 +282,7 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging { logInfo("Filtered files: \n" + generatedFiles.mkString("\n")) assert(generatedFiles.size > 1) val allText = generatedFiles.map { file => - Files.toString(file, UTF_8) + Files.toString(file, StandardCharsets.UTF_8) }.mkString("") assert(allText === expectedText) generatedFiles 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 412c0ac9d9be3..093d1bd6e5948 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -21,6 +21,7 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File, FileOutputStr import java.lang.{Double => JDouble, Float => JFloat} import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} +import java.nio.charset.StandardCharsets import java.text.DecimalFormatSymbols import java.util.Locale import java.util.concurrent.TimeUnit @@ -28,7 +29,6 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable.ListBuffer import scala.util.Random -import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration @@ -268,7 +268,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { val tmpDir2 = Utils.createTempDir() val f1Path = tmpDir2 + "/f1" val f1 = new FileOutputStream(f1Path) - f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(UTF_8)) + f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(StandardCharsets.UTF_8)) f1.close() // Read first few bytes @@ -295,9 +295,9 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { test("reading offset bytes across multiple files") { val tmpDir = Utils.createTempDir() val files = (1 to 3).map(i => new File(tmpDir, i.toString)) - Files.write("0123456789", files(0), UTF_8) - Files.write("abcdefghij", files(1), UTF_8) - Files.write("ABCDEFGHIJ", files(2), UTF_8) + Files.write("0123456789", files(0), StandardCharsets.UTF_8) + Files.write("abcdefghij", files(1), StandardCharsets.UTF_8) + Files.write("ABCDEFGHIJ", files(2), StandardCharsets.UTF_8) // Read first few bytes in the 1st file assert(Utils.offsetBytes(files, 0, 5) === "01234") @@ -529,7 +529,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { try { System.setProperty("spark.test.fileNameLoadB", "2") Files.write("spark.test.fileNameLoadA true\n" + - "spark.test.fileNameLoadB 1\n", outFile, UTF_8) + "spark.test.fileNameLoadB 1\n", outFile, StandardCharsets.UTF_8) val properties = Utils.getPropertiesFromFile(outFile.getAbsolutePath) properties .filter { case (k, v) => k.startsWith("spark.")} @@ -559,7 +559,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { val innerSourceDir = Utils.createTempDir(root = sourceDir.getPath) val sourceFile = File.createTempFile("someprefix", "somesuffix", innerSourceDir) val targetDir = new File(tempDir, "target-dir") - Files.write("some text", sourceFile, UTF_8) + Files.write("some text", sourceFile, StandardCharsets.UTF_8) val path = if (Utils.isWindows) { @@ -801,7 +801,7 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { |trap "" SIGTERM |sleep 10 """.stripMargin - Files.write(cmd.getBytes(), file) + Files.write(cmd.getBytes(StandardCharsets.UTF_8), file) file.getAbsoluteFile.setExecutable(true) val process = new ProcessBuilder(file.getAbsolutePath).start() diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 84547748618d1..732c83dc841d9 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -72,7 +72,8 @@ class CustomReceiver(host: String, port: Int) socket = new Socket(host, port) // Until stopped or connection broken continue reading - val reader = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8")) + val reader = new BufferedReader( + new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8)) userInput = reader.readLine() while(!isStopped && userInput != null) { store(userInput) @@ -135,7 +136,8 @@ public class JavaCustomReceiver extends Receiver { // connect to the server socket = new Socket(host, port); - BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream())); + BufferedReader reader = new BufferedReader( + new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8)); // Until stopped or connection broken continue reading while (!isStopped() && (userInput = reader.readLine()) != null) { diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java index 5de56340c6d22..4544ad2b42ca7 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java @@ -36,6 +36,7 @@ import java.io.InputStreamReader; import java.net.ConnectException; import java.net.Socket; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Iterator; import java.util.regex.Pattern; @@ -130,7 +131,8 @@ private void receive() { try { // connect to the server socket = new Socket(host, port); - reader = new BufferedReader(new InputStreamReader(socket.getInputStream())); + reader = new BufferedReader( + new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8)); // Until stopped or connection broken continue reading while (!isStopped() && (userInput = reader.readLine()) != null) { System.out.println("Received data '" + userInput + "'"); diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala index 5ce5778e42a3a..d67da270a8178 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala @@ -20,6 +20,7 @@ package org.apache.spark.examples.streaming import java.io.{BufferedReader, InputStreamReader} import java.net.Socket +import java.nio.charset.StandardCharsets import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.StorageLevel @@ -83,7 +84,8 @@ class CustomReceiver(host: String, port: Int) logInfo("Connecting to " + host + ":" + port) socket = new Socket(host, port) logInfo("Connected to " + host + ":" + port) - val reader = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8")) + val reader = new BufferedReader( + new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8)) userInput = reader.readLine() while(!isStopped && userInput != null) { store(userInput) diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala index 7f6cecf9cd18d..e8ca1e716394d 100644 --- a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala +++ b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.streaming.flume.sink import java.net.InetSocketAddress +import java.nio.charset.StandardCharsets import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} import java.util.concurrent.atomic.AtomicInteger @@ -184,7 +185,8 @@ class SparkSinkSuite extends FunSuite { private def putEvents(ch: MemoryChannel, count: Int): Unit = { val tx = ch.getTransaction tx.begin() - (1 to count).foreach(x => ch.put(EventBuilder.withBody(x.toString.getBytes))) + (1 to count).foreach(x => + ch.put(EventBuilder.withBody(x.toString.getBytes(StandardCharsets.UTF_8)))) tx.commit() tx.close() } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala index 3f87ce46e5952..945cfa7295d1d 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala @@ -19,12 +19,12 @@ package org.apache.spark.streaming.flume import java.net.{InetSocketAddress, ServerSocket} import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets import java.util.{List => JList} import java.util.Collections import scala.collection.JavaConverters._ -import com.google.common.base.Charsets.UTF_8 import org.apache.avro.ipc.NettyTransceiver import org.apache.avro.ipc.specific.SpecificRequestor import org.apache.commons.lang3.RandomUtils @@ -65,7 +65,7 @@ private[flume] class FlumeTestUtils { val inputEvents = input.asScala.map { item => val event = new AvroFlumeEvent - event.setBody(ByteBuffer.wrap(item.getBytes(UTF_8))) + event.setBody(ByteBuffer.wrap(item.getBytes(StandardCharsets.UTF_8))) event.setHeaders(Collections.singletonMap("test", "header")) event } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala index 9515d07c5ee5b..1a96df6e94b95 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala @@ -17,12 +17,12 @@ package org.apache.spark.streaming.flume +import java.nio.charset.StandardCharsets import java.util.{Collections, List => JList, Map => JMap} import java.util.concurrent._ import scala.collection.mutable.ArrayBuffer -import com.google.common.base.Charsets.UTF_8 import org.apache.flume.event.EventBuilder import org.apache.flume.Context import org.apache.flume.channel.MemoryChannel @@ -193,7 +193,8 @@ private[flume] class PollingFlumeTestUtils { val tx = channel.getTransaction tx.begin() for (j <- 0 until eventsPerBatch) { - channel.put(EventBuilder.withBody(s"${channel.getName}-$t".getBytes(UTF_8), + channel.put(EventBuilder.withBody( + s"${channel.getName}-$t".getBytes(StandardCharsets.UTF_8), Collections.singletonMap(s"test-$t", "header"))) t += 1 } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 0cb875c9758f9..72d9053355706 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -19,12 +19,12 @@ package org.apache.spark.streaming.kafka import java.io.OutputStream import java.lang.{Integer => JInt, Long => JLong} +import java.nio.charset.StandardCharsets import java.util.{List => JList, Map => JMap, Set => JSet} import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import com.google.common.base.Charsets.UTF_8 import kafka.common.TopicAndPartition import kafka.message.MessageAndMetadata import kafka.serializer.{Decoder, DefaultDecoder, StringDecoder} @@ -787,7 +787,7 @@ private object KafkaUtilsPythonHelper { def pickle(obj: Object, out: OutputStream, pickler: Pickler) { if (obj == this) { out.write(Opcodes.GLOBAL) - out.write(s"$module\nKafkaMessageAndMetadata\n".getBytes(UTF_8)) + out.write(s"$module\nKafkaMessageAndMetadata\n".getBytes(StandardCharsets.UTF_8)) } else { pickler.save(this) val msgAndMetaData = obj.asInstanceOf[PythonMessageAndMetadata] diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala index 0ace453ee9280..026387ed65d50 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.kinesis import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ @@ -242,7 +243,7 @@ private[kinesis] class SimpleDataGenerator( val shardIdToSeqNumbers = new mutable.HashMap[String, ArrayBuffer[(Int, String)]]() data.foreach { num => val str = num.toString - val data = ByteBuffer.wrap(str.getBytes()) + val data = ByteBuffer.wrap(str.getBytes(StandardCharsets.UTF_8)) val putRecordRequest = new PutRecordRequest().withStreamName(streamName) .withData(data) .withPartitionKey(str) diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala index fdb270eaad8c9..0b455e574e6fa 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KPLBasedKinesisTestUtils.scala @@ -17,6 +17,7 @@ package org.apache.spark.streaming.kinesis import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -51,7 +52,7 @@ private[kinesis] class KPLDataGenerator(regionName: String) extends KinesisDataG val shardIdToSeqNumbers = new mutable.HashMap[String, ArrayBuffer[(Int, String)]]() data.foreach { num => val str = num.toString - val data = ByteBuffer.wrap(str.getBytes()) + val data = ByteBuffer.wrap(str.getBytes(StandardCharsets.UTF_8)) val future = producer.addUserRecord(streamName, str, data) val kinesisCallBack = new FutureCallback[UserRecordResult]() { override def onFailure(t: Throwable): Unit = {} // do nothing diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala index 079bd8a9a87ea..cbad6f7fe44dc 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -17,6 +17,8 @@ package org.apache.spark.streaming.mqtt +import java.nio.charset.StandardCharsets + import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken import org.eclipse.paho.client.mqttv3.MqttCallback import org.eclipse.paho.client.mqttv3.MqttClient @@ -75,7 +77,7 @@ class MQTTReceiver( // Handles Mqtt message override def messageArrived(topic: String, message: MqttMessage) { - store(new String(message.getPayload(), "utf-8")) + store(new String(message.getPayload(), StandardCharsets.UTF_8)) } override def deliveryComplete(token: IMqttDeliveryToken) { diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTTestUtils.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTTestUtils.scala index 26c6dc45d5115..3680c136059a5 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTTestUtils.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTTestUtils.scala @@ -18,10 +18,10 @@ package org.apache.spark.streaming.mqtt import java.net.{ServerSocket, URI} +import java.nio.charset.StandardCharsets import scala.language.postfixOps -import com.google.common.base.Charsets.UTF_8 import org.apache.activemq.broker.{BrokerService, TransportConnector} import org.apache.commons.lang3.RandomUtils import org.eclipse.paho.client.mqttv3._ @@ -85,7 +85,7 @@ private[mqtt] class MQTTTestUtils extends Logging { client.connect() if (client.isConnected) { val msgTopic = client.getTopic(topic) - val message = new MqttMessage(data.getBytes(UTF_8)) + val message = new MqttMessage(data.getBytes(StandardCharsets.UTF_8)) message.setQos(1) message.setRetained(true) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphLoaderSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphLoaderSuite.scala index bff9f328d4907..e55b05fa996ad 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphLoaderSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphLoaderSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.graphx import java.io.File import java.io.FileOutputStream import java.io.OutputStreamWriter +import java.nio.charset.StandardCharsets import org.apache.spark.SparkFunSuite import org.apache.spark.util.Utils @@ -30,7 +31,7 @@ class GraphLoaderSuite extends SparkFunSuite with LocalSparkContext { withSpark { sc => val tmpDir = Utils.createTempDir() val graphFile = new File(tmpDir.getAbsolutePath, "graph.txt") - val writer = new OutputStreamWriter(new FileOutputStream(graphFile)) + val writer = new OutputStreamWriter(new FileOutputStream(graphFile), StandardCharsets.UTF_8) for (i <- (1 until 101)) writer.write(s"$i 0\n") writer.close() try { diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index 46410327a5d72..20387e0f1068d 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -23,6 +23,7 @@ import java.io.FileInputStream; import java.io.InputStreamReader; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -102,7 +103,7 @@ List buildJavaCommand(String extraClassPath) throws IOException { File javaOpts = new File(join(File.separator, getConfDir(), "java-opts")); if (javaOpts.isFile()) { BufferedReader br = new BufferedReader(new InputStreamReader( - new FileInputStream(javaOpts), "UTF-8")); + new FileInputStream(javaOpts), StandardCharsets.UTF_8)); try { String line; while ((line = br.readLine()) != null) { @@ -301,7 +302,7 @@ private Properties loadPropertiesFile() throws IOException { FileInputStream fd = null; try { fd = new FileInputStream(propsFile); - props.load(new InputStreamReader(fd, "UTF-8")); + props.load(new InputStreamReader(fd, StandardCharsets.UTF_8)); for (Map.Entry e : props.entrySet()) { e.setValue(e.getValue().toString().trim()); } diff --git a/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java b/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java index 6e7120167d605..c7959aee9f888 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java +++ b/launcher/src/main/java/org/apache/spark/launcher/OutputRedirector.java @@ -21,6 +21,7 @@ import java.io.InputStream; import java.io.InputStreamReader; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.concurrent.ThreadFactory; import java.util.logging.Level; import java.util.logging.Logger; @@ -42,7 +43,7 @@ class OutputRedirector { OutputRedirector(InputStream in, String loggerName, ThreadFactory tf) { this.active = true; - this.reader = new BufferedReader(new InputStreamReader(in)); + this.reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8)); this.thread = tf.newThread(new Runnable() { @Override public void run() { diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index d36731840b1a1..a85afb58b9c98 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -199,11 +199,7 @@ private void testCmdBuilder(boolean isDriver, boolean useDefaultPropertyFile) th for (String arg : cmd) { if (arg.startsWith("-XX:MaxPermSize=")) { - if (isDriver) { - assertEquals("-XX:MaxPermSize=256m", arg); - } else { - assertEquals("-XX:MaxPermSize=256m", arg); - } + assertEquals("-XX:MaxPermSize=256m", arg); } } 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 132dc174a894e..53935f328ab8a 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 @@ -19,6 +19,7 @@ package org.apache.spark.mllib.api.python import java.io.OutputStream import java.nio.{ByteBuffer, ByteOrder} +import java.nio.charset.StandardCharsets import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ @@ -1226,7 +1227,7 @@ private[spark] object SerDe extends Serializable { def pickle(obj: Object, out: OutputStream, pickler: Pickler): Unit = { if (obj == this) { out.write(Opcodes.GLOBAL) - out.write((module + "\n" + name + "\n").getBytes) + out.write((module + "\n" + name + "\n").getBytes(StandardCharsets.UTF_8)) } else { pickler.save(this) // it will be memorized by Pickler saveState(obj, out, pickler) diff --git a/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java b/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java index b8ddf907d05ad..1c18b2b266fef 100644 --- a/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/source/libsvm/JavaLibSVMRelationSuite.java @@ -19,8 +19,8 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; -import com.google.common.base.Charsets; import com.google.common.io.Files; import org.junit.After; @@ -55,7 +55,7 @@ public void setUp() throws IOException { tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "datasource"); File file = new File(tempDir, "part-00000"); String s = "1 1:1.0 3:2.0 5:3.0\n0\n0 2:4.0 4:5.0 6:6.0"; - Files.write(s, file, Charsets.US_ASCII); + Files.write(s, file, StandardCharsets.UTF_8); path = tempDir.toURI().toString(); } diff --git a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala index 84fc08be09ee7..71f4926290b20 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala @@ -17,9 +17,9 @@ package org.apache.spark.ml.source.libsvm -import java.io.{File, IOException} +import java.io.File +import java.nio.charset.StandardCharsets -import com.google.common.base.Charsets import com.google.common.io.Files import org.apache.spark.{SparkException, SparkFunSuite} @@ -42,7 +42,7 @@ class LibSVMRelationSuite extends SparkFunSuite with MLlibTestSparkContext { """.stripMargin tempDir = Utils.createTempDir() val file = new File(tempDir, "part-00000") - Files.write(lines, file, Charsets.US_ASCII) + Files.write(lines, file, StandardCharsets.UTF_8) path = tempDir.toURI.toString } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 70219e9ad9d3e..e542f21a1802c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -18,11 +18,11 @@ package org.apache.spark.mllib.util import java.io.File +import java.nio.charset.StandardCharsets import scala.io.Source import breeze.linalg.{squaredDistance => breezeSquaredDistance} -import com.google.common.base.Charsets import com.google.common.io.Files import org.apache.spark.SparkException @@ -84,7 +84,7 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { """.stripMargin val tempDir = Utils.createTempDir() val file = new File(tempDir.getPath, "part-00000") - Files.write(lines, file, Charsets.US_ASCII) + Files.write(lines, file, StandardCharsets.UTF_8) val path = tempDir.toURI.toString val pointsWithNumFeatures = loadLibSVMFile(sc, path, 6).collect() @@ -117,7 +117,7 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { """.stripMargin val tempDir = Utils.createTempDir() val file = new File(tempDir.getPath, "part-00000") - Files.write(lines, file, Charsets.US_ASCII) + Files.write(lines, file, StandardCharsets.UTF_8) val path = tempDir.toURI.toString intercept[SparkException] { @@ -134,7 +134,7 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { """.stripMargin val tempDir = Utils.createTempDir() val file = new File(tempDir.getPath, "part-00000") - Files.write(lines, file, Charsets.US_ASCII) + Files.write(lines, file, StandardCharsets.UTF_8) val path = tempDir.toURI.toString intercept[SparkException] { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/parser/ParseUtils.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/parser/ParseUtils.java index 2520c7bb8dae4..01f89112a759b 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/parser/ParseUtils.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/parser/ParseUtils.java @@ -18,6 +18,8 @@ package org.apache.spark.sql.catalyst.parser; +import java.nio.charset.StandardCharsets; + /** * A couple of utility methods that help with parsing ASTs. * @@ -76,7 +78,7 @@ public static String unescapeSQLString(String b) { byte bVal = (byte) ((i3 - '0') + ((i2 - '0') * 8) + ((i1 - '0') * 8 * 8)); byte[] bValArr = new byte[1]; bValArr[0] = bVal; - String tmp = new String(bValArr); + String tmp = new String(bValArr, StandardCharsets.UTF_8); sb.append(tmp); i += 3; continue; 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 a76517a89cc4a..e6804d096cd96 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,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import org.json4s.JsonAST._ @@ -109,7 +110,7 @@ object Literal { case DateType => create(0, DateType) case TimestampType => create(0L, TimestampType) case StringType => Literal("") - case BinaryType => Literal("".getBytes) + case BinaryType => Literal("".getBytes(StandardCharsets.UTF_8)) case CalendarIntervalType => Literal(new CalendarInterval(0, 0)) case arr: ArrayType => create(Array(), arr) case map: MapType => create(Map(), map) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index d9a9b6151a0be..b11365b297184 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst import java.io._ +import java.nio.charset.StandardCharsets import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{NumericType, StringType} @@ -118,7 +119,7 @@ package object util { val writer = new PrintWriter(out) t.printStackTrace(writer) writer.flush() - new String(out.toByteArray) + new String(out.toByteArray, StandardCharsets.UTF_8) } def stringOrNull(a: AnyRef): String = if (a == null) null else a.toString 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 124172bd66f19..450222d8cbba3 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import java.nio.charset.StandardCharsets + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -54,7 +56,7 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Literal.default(FloatType), 0.0f) checkEvaluation(Literal.default(DoubleType), 0.0) checkEvaluation(Literal.default(StringType), "") - checkEvaluation(Literal.default(BinaryType), "".getBytes) + checkEvaluation(Literal.default(BinaryType), "".getBytes(StandardCharsets.UTF_8)) checkEvaluation(Literal.default(DecimalType.USER_DEFAULT), Decimal(0)) checkEvaluation(Literal.default(DecimalType.SYSTEM_DEFAULT), Decimal(0)) checkEvaluation(Literal.default(DateType), DateTimeUtils.toJavaDate(0)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala index 4ad65db0977c7..fba5f53715039 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import java.nio.charset.StandardCharsets + import com.google.common.math.LongMath import org.apache.spark.SparkFunSuite @@ -440,7 +442,7 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Hex(Literal(100800200404L)), "177828FED4") checkEvaluation(Hex(Literal(-100800200404L)), "FFFFFFE887D7012C") checkEvaluation(Hex(Literal.create(null, BinaryType)), null) - checkEvaluation(Hex(Literal("helloHex".getBytes())), "68656C6C6F486578") + checkEvaluation(Hex(Literal("helloHex".getBytes(StandardCharsets.UTF_8))), "68656C6C6F486578") // scalastyle:off // Turn off scala style for non-ascii chars checkEvaluation(Hex(Literal("三重的".getBytes("UTF8"))), "E4B889E9878DE79A84") @@ -452,7 +454,7 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("unhex") { checkEvaluation(Unhex(Literal.create(null, StringType)), null) - checkEvaluation(Unhex(Literal("737472696E67")), "string".getBytes) + checkEvaluation(Unhex(Literal("737472696E67")), "string".getBytes(StandardCharsets.UTF_8)) checkEvaluation(Unhex(Literal("")), new Array[Byte](0)) checkEvaluation(Unhex(Literal("F")), Array[Byte](15)) checkEvaluation(Unhex(Literal("ff")), Array[Byte](-1)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala index 75131a6170222..60d50baf511d9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MiscFunctionsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import java.nio.charset.StandardCharsets + import org.apache.commons.codec.digest.DigestUtils import org.apache.spark.SparkFunSuite @@ -27,7 +29,8 @@ import org.apache.spark.sql.types._ class MiscFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("md5") { - checkEvaluation(Md5(Literal("ABC".getBytes)), "902fbdd2b1df0c4f70b4a5d23525e932") + checkEvaluation(Md5(Literal("ABC".getBytes(StandardCharsets.UTF_8))), + "902fbdd2b1df0c4f70b4a5d23525e932") checkEvaluation(Md5(Literal.create(Array[Byte](1, 2, 3, 4, 5, 6), BinaryType)), "6ac1e56bc78f031059be7be854522c4c") checkEvaluation(Md5(Literal.create(null, BinaryType)), null) @@ -35,27 +38,31 @@ class MiscFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("sha1") { - checkEvaluation(Sha1(Literal("ABC".getBytes)), "3c01bdbb26f358bab27f267924aa2c9a03fcfdb8") + checkEvaluation(Sha1(Literal("ABC".getBytes(StandardCharsets.UTF_8))), + "3c01bdbb26f358bab27f267924aa2c9a03fcfdb8") checkEvaluation(Sha1(Literal.create(Array[Byte](1, 2, 3, 4, 5, 6), BinaryType)), "5d211bad8f4ee70e16c7d343a838fc344a1ed961") checkEvaluation(Sha1(Literal.create(null, BinaryType)), null) - checkEvaluation(Sha1(Literal("".getBytes)), "da39a3ee5e6b4b0d3255bfef95601890afd80709") + checkEvaluation(Sha1(Literal("".getBytes(StandardCharsets.UTF_8))), + "da39a3ee5e6b4b0d3255bfef95601890afd80709") checkConsistencyBetweenInterpretedAndCodegen(Sha1, BinaryType) } test("sha2") { - checkEvaluation(Sha2(Literal("ABC".getBytes), Literal(256)), DigestUtils.sha256Hex("ABC")) + checkEvaluation(Sha2(Literal("ABC".getBytes(StandardCharsets.UTF_8)), Literal(256)), + DigestUtils.sha256Hex("ABC")) checkEvaluation(Sha2(Literal.create(Array[Byte](1, 2, 3, 4, 5, 6), BinaryType), Literal(384)), DigestUtils.sha384Hex(Array[Byte](1, 2, 3, 4, 5, 6))) // unsupported bit length checkEvaluation(Sha2(Literal.create(null, BinaryType), Literal(1024)), null) checkEvaluation(Sha2(Literal.create(null, BinaryType), Literal(512)), null) - checkEvaluation(Sha2(Literal("ABC".getBytes), Literal.create(null, IntegerType)), null) + checkEvaluation(Sha2(Literal("ABC".getBytes(StandardCharsets.UTF_8)), + Literal.create(null, IntegerType)), null) checkEvaluation(Sha2(Literal.create(null, BinaryType), Literal.create(null, IntegerType)), null) } test("crc32") { - checkEvaluation(Crc32(Literal("ABC".getBytes)), 2743272264L) + checkEvaluation(Crc32(Literal("ABC".getBytes(StandardCharsets.UTF_8))), 2743272264L) checkEvaluation(Crc32(Literal.create(Array[Byte](1, 2, 3, 4, 5, 6), BinaryType)), 2180413220L) checkEvaluation(Crc32(Literal.create(null, BinaryType)), null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 68545f33e5465..1265908182b3a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import org.scalatest.Matchers @@ -77,16 +78,16 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val row = new SpecificMutableRow(fieldTypes) row.setLong(0, 0) row.update(1, UTF8String.fromString("Hello")) - row.update(2, "World".getBytes) + row.update(2, "World".getBytes(StandardCharsets.UTF_8)) val unsafeRow: UnsafeRow = converter.apply(row) assert(unsafeRow.getSizeInBytes === 8 + (8 * 3) + - roundedSize("Hello".getBytes.length) + - roundedSize("World".getBytes.length)) + roundedSize("Hello".getBytes(StandardCharsets.UTF_8).length) + + roundedSize("World".getBytes(StandardCharsets.UTF_8).length)) assert(unsafeRow.getLong(0) === 0) assert(unsafeRow.getString(1) === "Hello") - assert(unsafeRow.getBinary(2) === "World".getBytes) + assert(unsafeRow.getBinary(2) === "World".getBytes(StandardCharsets.UTF_8)) } test("basic conversion with primitive, string, date and timestamp types") { @@ -100,7 +101,8 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { row.update(3, DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2015-05-08 08:10:25"))) val unsafeRow: UnsafeRow = converter.apply(row) - assert(unsafeRow.getSizeInBytes === 8 + (8 * 4) + roundedSize("Hello".getBytes.length)) + assert(unsafeRow.getSizeInBytes === + 8 + (8 * 4) + roundedSize("Hello".getBytes(StandardCharsets.UTF_8).length)) assert(unsafeRow.getLong(0) === 0) assert(unsafeRow.getString(1) === "Hello") @@ -175,7 +177,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { r.setFloat(6, 600) r.setDouble(7, 700) r.update(8, UTF8String.fromString("hello")) - r.update(9, "world".getBytes) + r.update(9, "world".getBytes(StandardCharsets.UTF_8)) r.setDecimal(10, Decimal(10), 10) r.setDecimal(11, Decimal(10.00, 38, 18), 38) // r.update(11, Array(11)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala index 1522ee34e43a5..e2a8eb8ee1d34 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions.codegen +import java.nio.charset.StandardCharsets + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ @@ -107,7 +109,8 @@ class GeneratedProjectionSuite extends SparkFunSuite { val fields = Array[DataType](StringType, struct) val unsafeProj = UnsafeProjection.create(fields) - val innerRow = InternalRow(false, 1.toByte, 2.toShort, 3, 4.0f, "".getBytes, + val innerRow = InternalRow(false, 1.toByte, 2.toShort, 3, 4.0f, + "".getBytes(StandardCharsets.UTF_8), UTF8String.fromString("")) val row1 = InternalRow(UTF8String.fromString(""), innerRow) val unsafe1 = unsafeProj(row1).copy() diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java index 68f146f7a2622..b084eda6f84c1 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java @@ -18,6 +18,7 @@ import java.math.BigDecimal; import java.math.BigInteger; +import java.nio.charset.StandardCharsets; import java.sql.Date; import java.util.Iterator; import java.util.List; @@ -138,7 +139,7 @@ private static void appendValue(ColumnVector dst, DataType t, Object o) { } else if (t == DataTypes.DoubleType) { dst.appendDouble(((Double)o).doubleValue()); } else if (t == DataTypes.StringType) { - byte[] b =((String)o).getBytes(); + byte[] b =((String)o).getBytes(StandardCharsets.UTF_8); dst.appendByteArray(b, 0, b.length); } else if (t instanceof DecimalType) { DecimalType dt = (DecimalType)t; diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java index 9a8aedfa56b8c..09c001baaeafd 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java @@ -18,6 +18,7 @@ import java.util.Arrays; import java.util.Iterator; +import java.util.NoSuchElementException; import org.apache.commons.lang.NotImplementedException; @@ -254,6 +255,9 @@ public Row next() { while (rowId < maxRows && ColumnarBatch.this.filteredRows[rowId]) { ++rowId; } + if (rowId >= maxRows) { + throw new NoSuchElementException(); + } row.rowId = rowId++; return row; } 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 38aa2dd80a4f1..6a0290c11228f 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.csv -import java.nio.charset.Charset +import java.nio.charset.StandardCharsets import org.apache.spark.Logging import org.apache.spark.sql.execution.datasources.CompressionCodecs @@ -64,7 +64,7 @@ private[sql] class CSVOptions( parameters.getOrElse("sep", parameters.getOrElse("delimiter", ","))) val parseMode = parameters.getOrElse("mode", "PERMISSIVE") val charset = parameters.getOrElse("encoding", - parameters.getOrElse("charset", Charset.forName("UTF-8").name())) + parameters.getOrElse("charset", StandardCharsets.UTF_8.name())) val quote = getChar("quote", '\"') val escape = getChar("escape", '\\') diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala index 8f1421844c648..8c3f63d307321 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources.csv import java.io.{ByteArrayOutputStream, OutputStreamWriter, StringReader} +import java.nio.charset.StandardCharsets import com.univocity.parsers.csv.{CsvParser, CsvParserSettings, CsvWriter, CsvWriterSettings} @@ -76,7 +77,7 @@ private[sql] class LineCsvWriter(params: CSVOptions, headers: Seq[String]) exten def writeRow(row: Seq[String], includeHeader: Boolean): String = { val buffer = new ByteArrayOutputStream() - val outputWriter = new OutputStreamWriter(buffer) + val outputWriter = new OutputStreamWriter(buffer, StandardCharsets.UTF_8) val writer = new CsvWriter(outputWriter, writerSettings) if (includeHeader) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala index aff672281d640..42c07c8a23f5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/DefaultSource.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.csv -import java.nio.charset.Charset +import java.nio.charset.{Charset, StandardCharsets} import org.apache.hadoop.fs.FileStatus import org.apache.hadoop.io.{LongWritable, Text} @@ -161,7 +161,7 @@ class DefaultSource extends FileFormat with DataSourceRegister { sqlContext: SQLContext, options: CSVOptions, location: String): RDD[String] = { - if (Charset.forName(options.charset) == Charset.forName("UTF-8")) { + if (Charset.forName(options.charset) == StandardCharsets.UTF_8) { sqlContext.sparkContext.textFile(location) } else { val charset = options.charset 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 14ba9f69bb1d7..cce4b74ff2994 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 @@ -18,11 +18,11 @@ package org.apache.spark.sql.execution.streaming import java.io._ +import java.nio.charset.StandardCharsets import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.io.Codec -import com.google.common.base.Charsets.UTF_8 import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.spark.Logging @@ -184,7 +184,7 @@ class FileStreamSource( private def writeBatch(id: Int, files: Seq[String]): Unit = { assert(files.nonEmpty, "create a new batch without any file") val output = fs.create(new Path(metadataPath + "/" + id), true) - val writer = new PrintWriter(new OutputStreamWriter(output, UTF_8)) + val writer = new PrintWriter(new OutputStreamWriter(output, StandardCharsets.UTF_8)) try { // scalastyle:off println writer.println(FileStreamSource.VERSION) diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java index 0f9e453d26db8..9e65158eb0a33 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/JavaSaveLoadSuite.java @@ -40,7 +40,6 @@ public class JavaSaveLoadSuite { private transient JavaSparkContext sc; private transient SQLContext sqlContext; - String originalDefaultSource; File path; Dataset df; @@ -57,7 +56,6 @@ public void setUp() throws IOException { sqlContext = new SQLContext(_sc); sc = new JavaSparkContext(_sc); - originalDefaultSource = sqlContext.conf().defaultDataSourceName(); path = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "datasource").getCanonicalFile(); if (path.exists()) { 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 aff9efe4b2b16..2aa6f8d4acf7d 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql +import java.nio.charset.StandardCharsets + import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -167,12 +169,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { } test("misc sha1 function") { - val df = Seq(("ABC", "ABC".getBytes)).toDF("a", "b") + val df = Seq(("ABC", "ABC".getBytes(StandardCharsets.UTF_8))).toDF("a", "b") checkAnswer( df.select(sha1($"a"), sha1($"b")), Row("3c01bdbb26f358bab27f267924aa2c9a03fcfdb8", "3c01bdbb26f358bab27f267924aa2c9a03fcfdb8")) - val dfEmpty = Seq(("", "".getBytes)).toDF("a", "b") + val dfEmpty = Seq(("", "".getBytes(StandardCharsets.UTF_8))).toDF("a", "b") checkAnswer( dfEmpty.selectExpr("sha1(a)", "sha1(b)"), Row("da39a3ee5e6b4b0d3255bfef95601890afd80709", "da39a3ee5e6b4b0d3255bfef95601890afd80709")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index e6e27ec413bb2..2333fa27ca623 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import java.io.File +import java.nio.charset.StandardCharsets import scala.language.postfixOps import scala.util.Random @@ -665,8 +666,8 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { test("showString: binary") { val df = Seq( - ("12".getBytes, "ABC.".getBytes), - ("34".getBytes, "12346".getBytes) + ("12".getBytes(StandardCharsets.UTF_8), "ABC.".getBytes(StandardCharsets.UTF_8)), + ("34".getBytes(StandardCharsets.UTF_8), "12346".getBytes(StandardCharsets.UTF_8)) ).toDF() val expectedAnswer = """+-------+----------------+ || _1| _2| diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala index 013a90875e2b2..f5a67fd782d63 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import java.nio.charset.StandardCharsets + import org.apache.spark.sql.functions._ import org.apache.spark.sql.functions.{log => logarithm} import org.apache.spark.sql.test.SharedSQLContext @@ -262,9 +264,9 @@ class MathExpressionsSuite extends QueryTest with SharedSQLContext { test("unhex") { val data = Seq(("1C", "737472696E67")).toDF("a", "b") checkAnswer(data.select(unhex('a)), Row(Array[Byte](28.toByte))) - checkAnswer(data.select(unhex('b)), Row("string".getBytes)) + checkAnswer(data.select(unhex('b)), Row("string".getBytes(StandardCharsets.UTF_8))) checkAnswer(data.selectExpr("unhex(a)"), Row(Array[Byte](28.toByte))) - checkAnswer(data.selectExpr("unhex(b)"), Row("string".getBytes)) + checkAnswer(data.selectExpr("unhex(b)"), Row("string".getBytes(StandardCharsets.UTF_8))) checkAnswer(data.selectExpr("""unhex("##")"""), Row(null)) checkAnswer(data.selectExpr("""unhex("G123")"""), Row(null)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala index 0000a5d1efd09..1aadd700d7443 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.columnar.compression import java.nio.{ByteBuffer, ByteOrder} +import java.nio.charset.StandardCharsets import org.apache.commons.lang3.RandomStringUtils import org.apache.commons.math3.distribution.LogNormalDistribution @@ -313,7 +314,7 @@ object CompressionSchemeBenchmark extends AllCompressionSchemes { } for (i <- 0 until count) { testData.putInt(strLen) - testData.put(g().getBytes) + testData.put(g().getBytes(StandardCharsets.UTF_8)) } testData.rewind() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala index 97638a66ab473..67b3d98c1daed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources.parquet import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets import scala.util.Random @@ -357,7 +358,8 @@ object ColumnarBatchBenchmark { val maxString = 32 val count = 4 * 1000 - val data = Seq.fill(count)(randomString(minString, maxString)).map(_.getBytes).toArray + val data = Seq.fill(count)(randomString(minString, maxString)) + .map(_.getBytes(StandardCharsets.UTF_8)).toArray def column(memoryMode: MemoryMode) = { i: Int => val column = ColumnVector.allocate(count, BinaryType, memoryMode) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index b3c3e66fbcbd5..ed97f59ea1690 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.vectorized +import java.nio.charset.StandardCharsets + import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.Random @@ -329,18 +331,21 @@ class ColumnarBatchSuite extends SparkFunSuite { var idx = 0 val values = ("Hello" :: "abc" :: Nil).toArray - column.putByteArray(idx, values(0).getBytes, 0, values(0).getBytes().length) + column.putByteArray(idx, values(0).getBytes(StandardCharsets.UTF_8), + 0, values(0).getBytes(StandardCharsets.UTF_8).length) reference += values(0) idx += 1 assert(column.arrayData().elementsAppended == 5) - column.putByteArray(idx, values(1).getBytes, 0, values(1).getBytes().length) + column.putByteArray(idx, values(1).getBytes(StandardCharsets.UTF_8), + 0, values(1).getBytes(StandardCharsets.UTF_8).length) reference += values(1) idx += 1 assert(column.arrayData().elementsAppended == 8) // Just put llo - val offset = column.putByteArray(idx, values(0).getBytes, 2, values(0).getBytes().length - 2) + val offset = column.putByteArray(idx, values(0).getBytes(StandardCharsets.UTF_8), + 2, values(0).getBytes(StandardCharsets.UTF_8).length - 2) reference += "llo" idx += 1 assert(column.arrayData().elementsAppended == 11) @@ -353,7 +358,7 @@ class ColumnarBatchSuite extends SparkFunSuite { // Put a long string val s = "abcdefghijklmnopqrstuvwxyz" - column.putByteArray(idx, (s + s).getBytes) + column.putByteArray(idx, (s + s).getBytes(StandardCharsets.UTF_8)) reference += (s + s) idx += 1 assert(column.arrayData().elementsAppended == 11 + (s + s).length) @@ -473,7 +478,7 @@ class ColumnarBatchSuite extends SparkFunSuite { batch.column(0).putInt(0, 1) batch.column(1).putDouble(0, 1.1) batch.column(2).putNull(0) - batch.column(3).putByteArray(0, "Hello".getBytes) + batch.column(3).putByteArray(0, "Hello".getBytes(StandardCharsets.UTF_8)) batch.setNumRows(1) // Verify the results of the row. @@ -519,17 +524,17 @@ class ColumnarBatchSuite extends SparkFunSuite { batch.column(0).putNull(0) batch.column(1).putDouble(0, 2.2) batch.column(2).putInt(0, 2) - batch.column(3).putByteArray(0, "abc".getBytes) + batch.column(3).putByteArray(0, "abc".getBytes(StandardCharsets.UTF_8)) batch.column(0).putInt(1, 3) batch.column(1).putNull(1) batch.column(2).putInt(1, 3) - batch.column(3).putByteArray(1, "".getBytes) + batch.column(3).putByteArray(1, "".getBytes(StandardCharsets.UTF_8)) batch.column(0).putInt(2, 4) batch.column(1).putDouble(2, 4.4) batch.column(2).putInt(2, 4) - batch.column(3).putByteArray(2, "world".getBytes) + batch.column(3).putByteArray(2, "world".getBytes(StandardCharsets.UTF_8)) batch.setNumRows(3) def rowEquals(x: InternalRow, y: Row): Unit = { 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 e9d77abb8c23c..e6889bcc783b6 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 @@ -18,8 +18,7 @@ package org.apache.spark.sql.streaming import java.io.{ByteArrayInputStream, File, FileNotFoundException, InputStream} - -import com.google.common.base.Charsets.UTF_8 +import java.nio.charset.StandardCharsets import org.apache.spark.sql.{AnalysisException, StreamTest} import org.apache.spark.sql.catalyst.util._ @@ -392,7 +391,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest with SharedSQLContext { } test("readBatch") { - def stringToStream(str: String): InputStream = new ByteArrayInputStream(str.getBytes(UTF_8)) + def stringToStream(str: String): InputStream = + new ByteArrayInputStream(str.getBytes(StandardCharsets.UTF_8)) // Invalid metadata assert(readBatch(stringToStream("")) === Nil) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala index 83c63e04f344a..7fa6760b71c8b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestData.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.test +import java.nio.charset.StandardCharsets + import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext, SQLImplicits} @@ -103,11 +105,11 @@ private[sql] trait SQLTestData { self => protected lazy val binaryData: DataFrame = { val df = sqlContext.sparkContext.parallelize( - BinaryData("12".getBytes, 1) :: - BinaryData("22".getBytes, 5) :: - BinaryData("122".getBytes, 3) :: - BinaryData("121".getBytes, 2) :: - BinaryData("123".getBytes, 4) :: Nil).toDF() + BinaryData("12".getBytes(StandardCharsets.UTF_8), 1) :: + BinaryData("22".getBytes(StandardCharsets.UTF_8), 5) :: + BinaryData("122".getBytes(StandardCharsets.UTF_8), 3) :: + BinaryData("121".getBytes(StandardCharsets.UTF_8), 2) :: + BinaryData("123".getBytes(StandardCharsets.UTF_8), 4) :: Nil).toDF() df.registerTempTable("binaryData") df } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 81508e134695a..54fffb971dbc3 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.thriftserver import java.io._ +import java.nio.charset.StandardCharsets import java.sql.Timestamp import java.util.Date @@ -121,7 +122,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { val process = new ProcessBuilder(command: _*).start() - val stdinWriter = new OutputStreamWriter(process.getOutputStream) + val stdinWriter = new OutputStreamWriter(process.getOutputStream, StandardCharsets.UTF_8) stdinWriter.write(queriesString) stdinWriter.flush() stdinWriter.close() diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index c05527b519daa..e89bb1c470d5a 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.thriftserver import java.io.File import java.net.URL +import java.nio.charset.StandardCharsets import java.sql.{Date, DriverManager, SQLException, Statement} import scala.collection.mutable @@ -28,7 +29,6 @@ import scala.concurrent.duration._ import scala.io.Source import scala.util.{Random, Try} -import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.jdbc.HiveDriver @@ -700,7 +700,7 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n """.stripMargin, new File(s"$tempLog4jConf/log4j.properties"), - UTF_8) + StandardCharsets.UTF_8) tempLog4jConf } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 081d849a88886..9725dcfde169c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive import java.io.File import java.net.{URL, URLClassLoader} +import java.nio.charset.StandardCharsets import java.sql.Timestamp import java.util.concurrent.TimeUnit import java.util.regex.Pattern @@ -715,7 +716,7 @@ private[hive] object HiveContext { case (null, _) => "NULL" case (d: Int, DateType) => new DateWritable(d).toString case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString - case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") + case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) case (decimal: java.math.BigDecimal, DecimalType()) => // Hive strips trailing zeros so use its toString HiveDecimal.create(decimal).toString diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 5e6641693798f..b6e2f1f6b3ab7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.execution import java.io._ +import java.nio.charset.StandardCharsets import java.util.Properties import javax.annotation.Nullable @@ -113,7 +114,7 @@ case class ScriptTransformation( ioschema.initOutputSerDe(output).getOrElse((null, null)) } - val reader = new BufferedReader(new InputStreamReader(inputStream)) + val reader = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8)) val outputIterator: Iterator[InternalRow] = new Iterator[InternalRow] with HiveInspectors { var curLine: String = null val scriptOutputStream = new DataInputStream(inputStream) diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java index 5a539eaec7507..e9356541c22df 100644 --- a/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/JavaMetastoreDataSourcesSuite.java @@ -48,7 +48,6 @@ public class JavaMetastoreDataSourcesSuite { private transient JavaSparkContext sc; private transient HiveContext sqlContext; - String originalDefaultSource; File path; Path hiveManagedPath; FileSystem fs; @@ -66,7 +65,6 @@ public void setUp() throws IOException { sqlContext = TestHive$.MODULE$; sc = new JavaSparkContext(sqlContext.sparkContext()); - originalDefaultSource = sqlContext.conf().defaultDataSourceName(); path = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "datasource").getCanonicalFile(); if (path.exists()) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala index 9ca07e96eb088..8cfb32f00a884 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.orc import java.io.File +import java.nio.charset.StandardCharsets import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.io.orc.CompressionKind @@ -73,7 +74,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest { test("Read/write binary data") { withOrcFile(BinaryData("test".getBytes("utf8")) :: Nil) { file => val bytes = read.orc(file).head().getAs[Array[Byte]](0) - assert(new String(bytes, "utf8") === "test") + assert(new String(bytes, StandardCharsets.UTF_8) === "test") } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index 441477479167a..f7519c10c8eb1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -19,6 +19,7 @@ package org.apache.spark.streaming.dstream import java.io._ import java.net.{ConnectException, Socket} +import java.nio.charset.StandardCharsets import scala.reflect.ClassTag import scala.util.control.NonFatal @@ -113,7 +114,8 @@ object SocketReceiver { * to '\n' delimited strings and returns an iterator to access the strings. */ def bytesToLines(inputStream: InputStream): Iterator[String] = { - val dataInputStream = new BufferedReader(new InputStreamReader(inputStream, "UTF-8")) + val dataInputStream = new BufferedReader( + new InputStreamReader(inputStream, StandardCharsets.UTF_8)) new NextIterator[String] { protected override def getNext() = { val nextValue = dataInputStream.readLine() diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 806cea24caddb..66448fd40057d 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -18,7 +18,7 @@ package org.apache.spark.streaming; import java.io.*; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.*; import java.util.concurrent.atomic.AtomicBoolean; @@ -1866,7 +1866,8 @@ public void testSocketString() { @Override public Iterable call(InputStream in) throws IOException { List out = new ArrayList<>(); - try (BufferedReader reader = new BufferedReader(new InputStreamReader(in))) { + try (BufferedReader reader = new BufferedReader( + new InputStreamReader(in, StandardCharsets.UTF_8))) { for (String line; (line = reader.readLine()) != null;) { out.add(line); } @@ -1930,7 +1931,7 @@ public void testRawSocketStream() { private static List> fileTestPrepare(File testDir) throws IOException { File existingFile = new File(testDir, "0"); - Files.write("0\n", existingFile, Charset.forName("UTF-8")); + Files.write("0\n", existingFile, StandardCharsets.UTF_8); Assert.assertTrue(existingFile.setLastModified(1000)); Assert.assertEquals(1000, existingFile.lastModified()); return Arrays.asList(Arrays.asList("0")); diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java index d09258e0e4a85..091ccbfd85cad 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java @@ -38,6 +38,7 @@ import java.io.Serializable; import java.net.ConnectException; import java.net.Socket; +import java.nio.charset.StandardCharsets; import java.util.concurrent.atomic.AtomicLong; public class JavaReceiverAPISuite implements Serializable { @@ -126,7 +127,8 @@ private void receive() { BufferedReader in = null; try { socket = new Socket(host, port); - in = new BufferedReader(new InputStreamReader(socket.getInputStream())); + in = new BufferedReader( + new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8)); String userInput; while ((userInput = in.readLine()) != null) { store(userInput); diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index ca716cf4e6f11..9a3248b3e8175 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -18,12 +18,12 @@ package org.apache.spark.streaming import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File, ObjectOutputStream} +import java.nio.charset.StandardCharsets import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.JavaConverters._ import scala.reflect.ClassTag -import com.google.common.base.Charsets import com.google.common.io.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} @@ -609,7 +609,7 @@ class CheckpointSuite extends TestSuiteBase with DStreamCheckpointTester */ def writeFile(i: Int, clock: Clock): Unit = { val file = new File(testDir, i.toString) - Files.write(i + "\n", file, Charsets.UTF_8) + Files.write(i + "\n", file, StandardCharsets.UTF_8) assert(file.setLastModified(clock.getTimeMillis())) // Check that the file's modification date is actually the value we wrote, since rounding or // truncation will break the test: diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index fa17b3a15c4b6..cc2a67187e710 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming import java.io.{BufferedWriter, File, OutputStreamWriter} import java.net.{ServerSocket, Socket, SocketException} -import java.nio.charset.Charset +import java.nio.charset.StandardCharsets import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger @@ -146,7 +146,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val testDir = Utils.createTempDir() // Create a file that exists before the StreamingContext is created: val existingFile = new File(testDir, "0") - Files.write("0\n", existingFile, Charset.forName("UTF-8")) + Files.write("0\n", existingFile, StandardCharsets.UTF_8) assert(existingFile.setLastModified(10000) && existingFile.lastModified === 10000) // Set up the streaming context and input streams @@ -369,7 +369,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val testDir = Utils.createTempDir() // Create a file that exists before the StreamingContext is created: val existingFile = new File(testDir, "0") - Files.write("0\n", existingFile, Charset.forName("UTF-8")) + Files.write("0\n", existingFile, StandardCharsets.UTF_8) assert(existingFile.setLastModified(10000) && existingFile.lastModified === 10000) // Set up the streaming context and input streams @@ -393,7 +393,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val input = Seq(1, 2, 3, 4, 5) input.foreach { i => val file = new File(testDir, i.toString) - Files.write(i + "\n", file, Charset.forName("UTF-8")) + Files.write(i + "\n", file, StandardCharsets.UTF_8) assert(file.setLastModified(clock.getTimeMillis())) assert(file.lastModified === clock.getTimeMillis()) logInfo("Created file " + file) @@ -448,7 +448,7 @@ class TestServer(portToBind: Int = 0) extends Logging { try { clientSocket.setTcpNoDelay(true) val outputStream = new BufferedWriter( - new OutputStreamWriter(clientSocket.getOutputStream)) + new OutputStreamWriter(clientSocket.getOutputStream, StandardCharsets.UTF_8)) while (clientSocket.isConnected) { val msg = queue.poll(100, TimeUnit.MILLISECONDS) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala index faa9c4f0cbd6a..6406d53f8941a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming import java.io.{File, IOException} -import java.nio.charset.Charset +import java.nio.charset.StandardCharsets import java.util.UUID import scala.collection.JavaConverters._ @@ -371,7 +371,7 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) val localFile = new File(localTestDir, (i + 1).toString) val hadoopFile = new Path(testDir, (i + 1).toString) val tempHadoopFile = new Path(testDir, ".tmp_" + (i + 1).toString) - Files.write(input(i) + "\n", localFile, Charset.forName("UTF-8")) + Files.write(input(i) + "\n", localFile, StandardCharsets.UTF_8) var tries = 0 var done = false while (!done && tries < maxTries) { 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 0b5ceb768cc86..1035056457830 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 @@ -21,6 +21,7 @@ import java.io.{ByteArrayInputStream, DataInputStream, File, FileOutputStream, I OutputStreamWriter} import java.net.{InetAddress, UnknownHostException, URI} import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets import java.util.{Properties, UUID} import java.util.zip.{ZipEntry, ZipOutputStream} @@ -29,7 +30,6 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map} import scala.util.{Failure, Success, Try} import scala.util.control.NonFatal -import com.google.common.base.Charsets.UTF_8 import com.google.common.base.Objects import com.google.common.io.Files import org.apache.hadoop.conf.Configuration @@ -619,7 +619,7 @@ private[spark] class Client( val props = new Properties() sparkConf.getAll.foreach { case (k, v) => props.setProperty(k, v) } confStream.putNextEntry(new ZipEntry(SPARK_CONF_FILE)) - val writer = new OutputStreamWriter(confStream, UTF_8) + val writer = new OutputStreamWriter(confStream, StandardCharsets.UTF_8) props.store(writer, "Spark configuration.") writer.flush() confStream.closeEntry() diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index b12e506033e39..78b57da482f70 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.yarn import java.io.{File, FileOutputStream, OutputStreamWriter} +import java.nio.charset.StandardCharsets import java.util.Properties import java.util.concurrent.TimeUnit @@ -25,7 +26,6 @@ import scala.collection.JavaConverters._ import scala.concurrent.duration._ import scala.language.postfixOps -import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.commons.lang3.SerializationUtils import org.apache.hadoop.yarn.conf.YarnConfiguration @@ -75,7 +75,7 @@ abstract class BaseYarnClusterSuite System.setProperty("SPARK_YARN_MODE", "true") val logConfFile = new File(logConfDir, "log4j.properties") - Files.write(LOG4J_CONF, logConfFile, UTF_8) + Files.write(LOG4J_CONF, logConfFile, StandardCharsets.UTF_8) // Disable the disk utilization check to avoid the test hanging when people's disks are // getting full. @@ -191,7 +191,7 @@ abstract class BaseYarnClusterSuite result: File, expected: String): Unit = { finalState should be (SparkAppHandle.State.FINISHED) - val resultString = Files.toString(result, UTF_8) + val resultString = Files.toString(result, StandardCharsets.UTF_8) resultString should be (expected) } @@ -231,7 +231,7 @@ abstract class BaseYarnClusterSuite extraConf.foreach { case (k, v) => props.setProperty(k, v) } val propsFile = File.createTempFile("spark", ".properties", tempDir) - val writer = new OutputStreamWriter(new FileOutputStream(propsFile), UTF_8) + val writer = new OutputStreamWriter(new FileOutputStream(propsFile), StandardCharsets.UTF_8) props.store(writer, "Spark properties.") writer.close() propsFile.getAbsolutePath() diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index e935163c3487f..5068c0cd208b6 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -19,13 +19,13 @@ package org.apache.spark.deploy.yarn import java.io.File import java.net.URL +import java.nio.charset.StandardCharsets import java.util.{HashMap => JHashMap} import scala.collection.mutable import scala.concurrent.duration._ import scala.language.postfixOps -import com.google.common.base.Charsets.UTF_8 import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.scalatest.Matchers @@ -147,7 +147,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { private def testPySpark(clientMode: Boolean): Unit = { val primaryPyFile = new File(tempDir, "test.py") - Files.write(TEST_PYFILE, primaryPyFile, UTF_8) + Files.write(TEST_PYFILE, primaryPyFile, StandardCharsets.UTF_8) // When running tests, let's not assume the user has built the assembly module, which also // creates the pyspark archive. Instead, let's use PYSPARK_ARCHIVES_PATH to point at the @@ -171,7 +171,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { subdir } val pyModule = new File(moduleDir, "mod1.py") - Files.write(TEST_PYMODULE, pyModule, UTF_8) + Files.write(TEST_PYMODULE, pyModule, StandardCharsets.UTF_8) val mod2Archive = TestUtils.createJarWithFiles(Map("mod2.py" -> TEST_PYMODULE), moduleDir) val pyFiles = Seq(pyModule.getAbsolutePath(), mod2Archive.getPath()).mkString(",") @@ -245,7 +245,7 @@ private object YarnClusterDriver extends Logging with Matchers { data should be (Set(1, 2, 3, 4)) result = "success" } finally { - Files.write(result, status, UTF_8) + Files.write(result, status, StandardCharsets.UTF_8) sc.stop() } @@ -319,14 +319,14 @@ private object YarnClasspathTest extends Logging { val ccl = Thread.currentThread().getContextClassLoader() val resource = ccl.getResourceAsStream("test.resource") val bytes = ByteStreams.toByteArray(resource) - result = new String(bytes, 0, bytes.length, UTF_8) + result = new String(bytes, 0, bytes.length, StandardCharsets.UTF_8) } catch { case t: Throwable => error(s"loading test.resource to $resultPath", t) // set the exit code if not yet set exitCode = 2 } finally { - Files.write(result, new File(resultPath), UTF_8) + Files.write(result, new File(resultPath), StandardCharsets.UTF_8) } } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index c17e8695c24fb..1538ff75be5b3 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.deploy.yarn import java.io.File +import java.nio.charset.StandardCharsets -import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.commons.io.FileUtils import org.apache.hadoop.yarn.conf.YarnConfiguration @@ -104,7 +104,7 @@ private object YarnExternalShuffleDriver extends Logging with Matchers { } finally { sc.stop() FileUtils.deleteDirectory(execStateCopy) - Files.write(result, status, UTF_8) + Files.write(result, status, StandardCharsets.UTF_8) } } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index 9202bd892f01b..70b8732946a2b 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.yarn import java.io.{File, IOException} import java.lang.reflect.InvocationTargetException +import java.nio.charset.StandardCharsets import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.conf.Configuration @@ -59,7 +60,7 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging val args = Array("arg1", "${arg.2}", "\"arg3\"", "'arg4'", "$arg5", "\\arg6") try { val argLine = args.map(a => YarnSparkHadoopUtil.escapeForShell(a)).mkString(" ") - Files.write(("bash -c \"echo " + argLine + "\"").getBytes(), scriptFile) + Files.write(("bash -c \"echo " + argLine + "\"").getBytes(StandardCharsets.UTF_8), scriptFile) scriptFile.setExecutable(true) val proc = Runtime.getRuntime().exec(Array(scriptFile.getAbsolutePath())) From e58fa19d17db5dd8a00551e20b46921f98b958f7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 13 Mar 2016 22:14:59 -0700 Subject: [PATCH 17/21] Closes #11668 From acdf21970334cea9d6cfc287e4ccb8e72de9dee1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 14 Mar 2016 09:07:39 +0000 Subject: [PATCH 18/21] [MINOR][DOCS] Fix more typos in comments/strings. ## What changes were proposed in this pull request? This PR fixes 135 typos over 107 files: * 121 typos in comments * 11 typos in testcase name * 3 typos in log messages ## How was this patch tested? Manual. Author: Dongjoon Hyun Closes #11689 from dongjoon-hyun/fix_more_typos. --- .../scala/org/apache/spark/api/java/JavaDoubleRDD.scala | 2 +- .../scala/org/apache/spark/api/java/JavaSparkContext.scala | 4 ++-- .../scala/org/apache/spark/broadcast/TorrentBroadcast.scala | 2 +- .../org/apache/spark/deploy/history/ApplicationCache.scala | 6 +++--- .../src/main/scala/org/apache/spark/executor/Executor.scala | 2 +- .../scala/org/apache/spark/rdd/DoubleRDDFunctions.scala | 2 +- .../scala/org/apache/spark/scheduler/InputFormatInfo.scala | 2 +- .../main/scala/org/apache/spark/scheduler/SplitInfo.scala | 2 +- .../main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- .../apache/spark/storage/ShuffleBlockFetcherIterator.scala | 2 +- .../scala/org/apache/spark/ui/UIWorkloadGenerator.scala | 2 +- .../scala/org/apache/spark/util/ParentClassLoader.scala | 2 +- .../main/scala/org/apache/spark/util/SizeEstimator.scala | 2 +- core/src/main/scala/org/apache/spark/util/Utils.scala | 4 ++-- .../scala/org/apache/spark/util/random/RandomSampler.scala | 4 ++-- .../scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 2 +- .../spark/deploy/StandaloneDynamicAllocationSuite.scala | 2 +- .../org/apache/spark/deploy/client/AppClientSuite.scala | 2 +- .../apache/spark/deploy/history/ApplicationCacheSuite.scala | 6 +++--- .../org/apache/spark/metrics/InputOutputMetricsSuite.scala | 6 +++--- .../test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala | 4 ++-- .../scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala | 2 +- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../org/apache/spark/scheduler/TaskSetManagerSuite.scala | 2 +- .../spark/serializer/SerializationDebuggerSuite.scala | 2 +- .../org/apache/spark/util/random/XORShiftRandomSuite.scala | 2 +- .../scala/org/apache/spark/examples/SkewedGroupByTest.scala | 2 +- .../org/apache/spark/streaming/flume/sink/Logging.scala | 2 +- .../apache/spark/streaming/flume/FlumeBatchFetcher.scala | 6 +++--- .../org/apache/spark/streaming/kafka/KafkaCluster.scala | 2 +- .../spark/streaming/kinesis/KinesisRecordProcessor.scala | 2 +- .../streaming/kinesis/KinesisBackedBlockRDDSuite.scala | 2 +- .../spark/streaming/kinesis/KinesisReceiverSuite.scala | 2 +- graphx/src/main/scala/org/apache/spark/graphx/Graph.scala | 2 +- .../src/main/scala/org/apache/spark/graphx/GraphOps.scala | 4 ++-- .../apache/spark/graphx/impl/ShippableVertexPartition.scala | 2 +- .../main/scala/org/apache/spark/graphx/lib/PageRank.scala | 4 ++-- graphx/src/main/scala/org/apache/spark/graphx/package.scala | 2 +- .../scala/org/apache/spark/graphx/util/BytecodeUtils.scala | 2 +- .../src/test/scala/org/apache/spark/graphx/GraphSuite.scala | 2 +- mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala | 2 +- .../scala/org/apache/spark/ml/attribute/attributes.scala | 2 +- .../org/apache/spark/mllib/evaluation/RankingMetrics.scala | 2 +- .../scala/org/apache/spark/mllib/fpm/AssociationRules.scala | 2 +- .../org/apache/spark/mllib/util/LinearDataGenerator.scala | 2 +- .../apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala | 2 +- .../mllib/classification/LogisticRegressionSuite.scala | 2 +- .../org/apache/spark/mllib/stat/HypothesisTestSuite.scala | 2 +- .../org/apache/spark/mllib/tree/DecisionTreeSuite.scala | 2 +- project/MimaExcludes.scala | 6 +++--- sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala | 2 +- .../org/apache/spark/sql/catalyst/ScalaReflection.scala | 2 +- .../spark/sql/catalyst/analysis/DecimalPrecision.scala | 2 +- .../sql/catalyst/analysis/DistinctAggregationRewriter.scala | 2 +- .../spark/sql/catalyst/analysis/HiveTypeCoercion.scala | 2 +- .../spark/sql/catalyst/expressions/AttributeSet.scala | 2 +- .../spark/sql/catalyst/expressions/ExpressionSet.scala | 2 +- .../apache/spark/sql/catalyst/expressions/ScalaUDF.scala | 2 +- .../sql/catalyst/expressions/codegen/CodeGenerator.scala | 4 ++-- .../sql/catalyst/expressions/conditionalExpressions.scala | 2 +- .../org/apache/spark/sql/catalyst/expressions/objects.scala | 2 +- .../apache/spark/sql/catalyst/util/NumberConverter.scala | 2 +- .../spark/sql/catalyst/analysis/AnalysisErrorSuite.scala | 2 +- .../apache/spark/sql/catalyst/analysis/AnalysisSuite.scala | 2 +- .../spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala | 2 +- .../spark/sql/catalyst/expressions/ExpressionSetSuite.scala | 2 +- .../expressions/aggregate/HyperLogLogPlusPlusSuite.scala | 2 +- sql/core/src/main/scala/org/apache/spark/sql/Column.scala | 2 +- .../main/scala/org/apache/spark/sql/GroupedDataset.scala | 2 +- .../scala/org/apache/spark/sql/execution/SparkPlan.scala | 4 ++-- .../apache/spark/sql/execution/columnar/ColumnType.scala | 2 +- .../spark/sql/execution/datasources/PartitioningUtils.scala | 2 +- .../spark/sql/execution/datasources/jdbc/JdbcUtils.scala | 2 +- .../spark/sql/execution/python/BatchPythonEvaluation.scala | 2 +- .../org/apache/spark/sql/execution/streaming/Sink.scala | 2 +- .../org/apache/spark/sql/execution/streaming/memory.scala | 2 +- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- .../scala/org/apache/spark/sql/DataFrameStatSuite.scala | 2 +- .../src/test/scala/org/apache/spark/sql/DatasetSuite.scala | 2 +- .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 2 +- .../src/test/scala/org/apache/spark/sql/StreamTest.scala | 4 ++-- .../sql/execution/datasources/csv/CSVInferSchemaSuite.scala | 2 +- .../test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 4 ++-- .../scala/org/apache/spark/sql/sources/TableScanSuite.scala | 2 +- .../org/apache/spark/sql/hive/thriftserver/CliSuite.scala | 2 +- .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 4 ++-- .../org/apache/spark/sql/hive/orc/OrcFileOperator.scala | 2 +- .../spark/sql/hive/execution/HiveComparisonTest.scala | 2 +- .../apache/spark/sql/hive/execution/HiveQuerySuite.scala | 2 +- .../main/scala/org/apache/spark/streaming/Checkpoint.scala | 2 +- .../scala/org/apache/spark/streaming/StreamingContext.scala | 2 +- .../apache/spark/streaming/api/java/JavaPairDStream.scala | 2 +- .../spark/streaming/api/java/JavaStreamingContext.scala | 2 +- .../spark/streaming/dstream/PairDStreamFunctions.scala | 4 ++-- .../apache/spark/streaming/dstream/TransformedDStream.scala | 2 +- .../spark/streaming/ui/StreamingJobProgressListener.scala | 2 +- .../scala/org/apache/spark/streaming/util/StateMap.scala | 2 +- .../org/apache/spark/streaming/MapWithStateSuite.scala | 2 +- .../apache/spark/streaming/ReceivedBlockHandlerSuite.scala | 6 +++--- .../apache/spark/streaming/ReceiverInputDStreamSuite.scala | 2 +- .../scala/org/apache/spark/streaming/StateMapSuite.scala | 2 +- .../org/apache/spark/streaming/StreamingContextSuite.scala | 2 +- .../scala/org/apache/spark/streaming/TestSuiteBase.scala | 2 +- .../apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala | 2 +- .../org/apache/spark/deploy/yarn/ExecutorRunnable.scala | 6 +++--- .../org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala | 6 +++--- .../spark/deploy/yarn/YarnShuffleIntegrationSuite.scala | 2 +- 107 files changed, 135 insertions(+), 135 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 13e18a56c8fd8..0d3a5237d9906 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -66,7 +66,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) */ def unpersist(blocking: Boolean): JavaDoubleRDD = fromRDD(srdd.unpersist(blocking)) - // first() has to be overriden here in order for its return type to be Double instead of Object. + // first() has to be overridden here in order for its return type to be Double instead of Object. override def first(): JDouble = srdd.first() // Transformations (return a new RDD) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index f1aebbcd39638..d362c40b7af4b 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -351,7 +351,7 @@ class JavaSparkContext(val sc: SparkContext) } /** - * Get an RDD for a Hadoop-readable dataset from a Hadooop JobConf giving its InputFormat and any + * Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf giving its InputFormat and any * other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable, * etc). * @@ -383,7 +383,7 @@ class JavaSparkContext(val sc: SparkContext) } /** - * Get an RDD for a Hadoop-readable dataset from a Hadooop JobConf giving its InputFormat and any + * Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf giving its InputFormat and any * other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable, * * @param conf JobConf for setting up the dataset. Note: This will be put into a Broadcast. diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 550e1ba6d3de0..8091aa8062a21 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -74,7 +74,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) } else { None } - // Note: use getSizeAsKb (not bytes) to maintain compatiblity if no units are provided + // Note: use getSizeAsKb (not bytes) to maintain compatibility if no units are provided blockSize = conf.getSizeAsKb("spark.broadcast.blockSize", "4m").toInt * 1024 } setConf(SparkEnv.get.conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala index e2fda29044385..000f7e8e1e6e8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationCache.scala @@ -87,7 +87,7 @@ private[history] class ApplicationCache( /** * The cache of applications. * - * Tagged as `protected` so as to allow subclasses in tests to accesss it directly + * Tagged as `protected` so as to allow subclasses in tests to access it directly */ protected val appCache: LoadingCache[CacheKey, CacheEntry] = { CacheBuilder.newBuilder() @@ -447,7 +447,7 @@ private[history] class CacheMetrics(prefix: String) extends Source { private[history] trait ApplicationCacheOperations { /** - * Get the application UI and the probe neededed to see if it has been updated. + * Get the application UI and the probe needed to see if it has been updated. * @param appId application ID * @param attemptId attempt ID * @return If found, the Spark UI and any history information to be used in the cache @@ -590,7 +590,7 @@ private[history] object ApplicationCacheCheckFilterRelay extends Logging { // name of the attempt ID entry in the filter configuration. Optional. val ATTEMPT_ID = "attemptId" - // namer of the filter to register + // name of the filter to register val FILTER_NAME = "org.apache.spark.deploy.history.ApplicationCacheCheckFilter" /** the application cache to relay requests to */ diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 07e3c12bc9bc3..48372d70d52a9 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -124,7 +124,7 @@ private[spark] class Executor( private val HEARTBEAT_MAX_FAILURES = conf.getInt("spark.executor.heartbeat.maxFailures", 60) /** - * Count the failure times of heartbeat. It should only be acessed in the heartbeat thread. Each + * Count the failure times of heartbeat. It should only be accessed in the heartbeat thread. Each * successful heartbeat will reset it to 0. */ private var heartbeatFailures = 0 diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index c9606600ed068..0f579cfe420c5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -141,7 +141,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * And on the input of 1 and 50 we would have a histogram of 1, 0, 1 * * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched - * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets + * from an O(log n) insertion to O(1) per element. (where n = # buckets) if you set evenBuckets * to true. * buckets must be sorted and not contain any duplicates. * buckets array must be at least two elements diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index def0aac720b64..dfcdd113dfb98 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -157,7 +157,7 @@ object InputFormatInfo { b) Decrement the currently allocated containers on that host. c) Compute rack info for each host and update rack -> count map based on (b). d) Allocate nodes based on (c) - e) On the allocation result, ensure that we dont allocate "too many" jobs on a single node + e) On the allocation result, ensure that we don't allocate "too many" jobs on a single node (even if data locality on that is very high) : this is to prevent fragility of job if a single (or small set of) hosts go down. diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index 1ce83485f024b..6e9337bb90635 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -45,7 +45,7 @@ class SplitInfo( hashCode } - // This is practically useless since most of the Split impl's dont seem to implement equals :-( + // This is practically useless since most of the Split impl's don't seem to implement equals :-( // So unless there is identity equality between underlyingSplits, it will always fail even if it // is pointing to same block. override def equals(other: Any): Boolean = other match { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index bcf65e9d7e25c..996c1f5d9e261 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1057,7 +1057,7 @@ private[spark] class BlockManager( failures += 1 replicationFailed = true peersFailedToReplicateTo += peer - if (failures > maxReplicationFailures) { // too many failures in replcating to peers + if (failures > maxReplicationFailures) { // too many failures in replicating to peers done = true } } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 478a928acd03c..b19c30e2ff779 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -70,7 +70,7 @@ final class ShuffleBlockFetcherIterator( private[this] var numBlocksToFetch = 0 /** - * The number of blocks proccessed by the caller. The iterator is exhausted when + * The number of blocks processed by the caller. The iterator is exhausted when * [[numBlocksProcessed]] == [[numBlocksToFetch]]. */ private[this] var numBlocksProcessed = 0 diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index 5a8c2914314c2..094953f2f5b5e 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -102,7 +102,7 @@ private[spark] object UIWorkloadGenerator { try { setProperties(desc) job() - println("Job funished: " + desc) + println("Job finished: " + desc) } catch { case e: Exception => println("Job Failed: " + desc) diff --git a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala index 73d126ff6254e..c9b7493fcdc1b 100644 --- a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala @@ -18,7 +18,7 @@ package org.apache.spark.util /** - * A class loader which makes some protected methods in ClassLoader accesible. + * A class loader which makes some protected methods in ClassLoader accessible. */ private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) { diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index 83ded92609d66..a06db9a4fcfa5 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -253,7 +253,7 @@ object SizeEstimator extends Logging { } else { // Estimate the size of a large array by sampling elements without replacement. // To exclude the shared objects that the array elements may link, sample twice - // and use the min one to caculate array size. + // and use the min one to calculate array size. val rand = new Random(42) val drawn = new OpenHashSet[Int](2 * ARRAY_SAMPLE_SIZE) val s1 = sampleArray(array, state, rand, drawn, length) 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 b5a98ce569a8a..37c6c9bf90b79 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1530,7 +1530,7 @@ private[spark] object Utils extends Logging { rawMod + (if (rawMod < 0) mod else 0) } - // Handles idiosyncracies with hash (add more as required) + // Handles idiosyncrasies with hash (add more as required) // This method should be kept in sync with // org.apache.spark.network.util.JavaUtils#nonNegativeHash(). def nonNegativeHash(obj: AnyRef): Int = { @@ -1600,7 +1600,7 @@ private[spark] object Utils extends Logging { * @param f function to be executed. If prepare is not None, the running time of each call to f * must be an order of magnitude longer than one millisecond for accurate timing. * @param prepare function to be executed before each call to f. Its running time doesn't count. - * @return the total time across all iterations (not couting preparation time) + * @return the total time across all iterations (not counting preparation time) */ def timeIt(numIters: Int)(f: => Unit, prepare: Option[() => Unit] = None): Long = { if (prepare.isEmpty) { diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index 1314217023d15..3c61528ab5287 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -54,7 +54,7 @@ object RandomSampler { /** * Default maximum gap-sampling fraction. * For sampling fractions <= this value, the gap sampling optimization will be applied. - * Above this value, it is assumed that "tradtional" Bernoulli sampling is faster. The + * Above this value, it is assumed that "traditional" Bernoulli sampling is faster. The * optimal value for this will depend on the RNG. More expensive RNGs will tend to make * the optimal value higher. The most reliable way to determine this value for a new RNG * is to experiment. When tuning for a new RNG, I would expect a value of 0.5 to be close @@ -319,7 +319,7 @@ class GapSamplingReplacementIterator[T: ClassTag]( /** * Skip elements with replication factor zero (i.e. elements that won't be sampled). * Samples 'k' from geometric distribution P(k) = (1-q)(q)^k, where q = e^(-f), that is - * q is the probabililty of Poisson(0; f) + * q is the probability of Poisson(0; f) */ private def advance(): Unit = { val u = math.max(rng.nextDouble(), epsilon) diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 91fef772d13c3..bb2adff57e944 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate import org.apache.spark.util.{ResetSystemProperties, Utils} // Note: this suite mixes in ResetSystemProperties because SparkSubmit.main() sets a bunch -// of properties that neeed to be cleared after tests. +// of properties that needed to be cleared after tests. class SparkSubmitSuite extends SparkFunSuite with Matchers diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index b7ff5c9e8c0d3..d2e24912b570f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -501,7 +501,7 @@ class StandaloneDynamicAllocationSuite master.self.askWithRetry[MasterStateResponse](RequestMasterState) } - /** Get the applictions that are active from Master */ + /** Get the applications that are active from Master */ private def getApplications(): Seq[ApplicationInfo] = { getMasterState.activeApps } diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala index 379c038c5503d..7017296bd1361 100644 --- a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala @@ -159,7 +159,7 @@ class AppClientSuite extends SparkFunSuite with LocalSparkContext with BeforeAnd master.self.askWithRetry[MasterStateResponse](RequestMasterState) } - /** Get the applictions that are active from Master */ + /** Get the applications that are active from Master */ private def getApplications(): Seq[ApplicationInfo] = { getMasterState.activeApps } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala index e24188781f7cd..c874b95b0960a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala @@ -219,7 +219,7 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar val cacheEntry = cache.lookupCacheEntry(app1, None) assert(1 === cacheEntry.probeTime) assert(cacheEntry.completed) - // assert about queries made of the opereations + // assert about queries made of the operations assert(1 === operations.getAppUICount, "getAppUICount") assert(1 === operations.attachCount, "attachCount") @@ -338,7 +338,7 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar } /** - * Look up the cache entry and assert that it maches in the expected value. + * Look up the cache entry and assert that it matches in the expected value. * This assertion works if the two CacheEntries are different -it looks at the fields. * UI are compared on object equality; the timestamp and completed flags directly. * @param appId application ID @@ -384,7 +384,7 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar val operations = new StubCacheOperations() val clock = new ManualClock(0) val size = 5 - // only two entries are retained, so we expect evictions to occurr on lookups + // only two entries are retained, so we expect evictions to occur on lookups implicit val cache: ApplicationCache = new TestApplicationCache(operations, retainedApplications = size, clock = clock) diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index e5a448298a624..056e5463a0abf 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -98,14 +98,14 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext rdd.coalesce(4).count() } - // for count and coelesce, the same bytes should be read. + // for count and coalesce, the same bytes should be read. assert(bytesRead != 0) assert(bytesRead2 == bytesRead) } /** * This checks the situation where we have interleaved reads from - * different sources. Currently, we only accumulate fron the first + * different sources. Currently, we only accumulate from the first * read method we find in the task. This test uses cartesian to create * the interleaved reads. * @@ -183,7 +183,7 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext assert(records == numRecords) } - test("input metrics on recordsd read with cache") { + test("input metrics on records read with cache") { // prime the cache manager val rdd = sc.textFile(tmpFilePath, 4).cache() rdd.collect() diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index 76451788d2406..864adddad3426 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -164,8 +164,8 @@ class DoubleRDDSuite extends SparkFunSuite with SharedSparkContext { val expectedHistogramResults = Array(4, 2, 1, 2, 3) assert(histogramResults === expectedHistogramResults) } - // Make sure this works with a NaN end bucket and an inifity - test("WorksMixedRangeWithUnevenBucketsAndNaNAndNaNRangeAndInfity") { + // Make sure this works with a NaN end bucket and an infinity + test("WorksMixedRangeWithUnevenBucketsAndNaNAndNaNRangeAndInfinity") { // Make sure that it works with two unequally spaced buckets and elements in each val rdd = sc.parallelize(Seq(-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1, 1.0/0.0, -1.0/0.0, Double.NaN)) diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 7d51538d92597..b0d69de6e2ef4 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -182,7 +182,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { assert(sums(2) === 1) } - test("reduceByKey with many output partitons") { + test("reduceByKey with many output partitions") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) val sums = pairs.reduceByKey(_ + _, 10).collect() assert(sums.toSet === Set((1, 7), (2, 1))) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index d8849d59482e6..d1c7143abf834 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -663,7 +663,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou completeShuffleMapStageSuccessfully(0, 1, numShufflePartitions = parts) completeNextResultStageWithSuccess(1, 1) - // Confirm job finished succesfully + // Confirm job finished successfully sc.listenerBus.waitUntilEmpty(1000) assert(ended === true) assert(results === (0 until parts).map { idx => idx -> 42 }.toMap) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 2c99dd5afb32e..d35ca411f4080 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -396,7 +396,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val rescheduleDelay = 300L val conf = new SparkConf(). set("spark.scheduler.executorTaskBlacklistTime", rescheduleDelay.toString). - // dont wait to jump locality levels in this test + // don't wait to jump locality levels in this test set("spark.locality.wait", "0") sc = new SparkContext("local", "test", conf) diff --git a/core/src/test/scala/org/apache/spark/serializer/SerializationDebuggerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/SerializationDebuggerSuite.scala index 683aaa3aab1ba..bdee889cdc409 100644 --- a/core/src/test/scala/org/apache/spark/serializer/SerializationDebuggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/SerializationDebuggerSuite.scala @@ -191,7 +191,7 @@ class SerializationDebuggerSuite extends SparkFunSuite with BeforeAndAfterEach { } val originalException = new NotSerializableException("someClass") - // verify thaht original exception is returned on failure + // verify that original exception is returned on failure assert(SerializationDebugger.improveException(o, originalException).eq(originalException)) } } diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index 853503bbc2bba..83eba3690e289 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -53,7 +53,7 @@ class XORShiftRandomSuite extends SparkFunSuite with Matchers { * Perform the chi square test on the 5 rows of randomly generated numbers evenly divided into * 10 bins. chiSquareTest returns true iff the null hypothesis (that the classifications * represented by the counts in the columns of the input 2-way table are independent of the - * rows) can be rejected with 100 * (1 - alpha) percent confidence, where alpha is prespeficied + * rows) can be rejected with 100 * (1 - alpha) percent confidence, where alpha is prespecified * as 0.05 */ val chiTest = new ChiSquareTest diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 7796f362bb56c..d498af9c390a7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -38,7 +38,7 @@ object SkewedGroupByTest { val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random - // map output sizes lineraly increase from the 1st to the last + // map output sizes linearly increase from the 1st to the last numKVPairs = (1.0 * (p + 1) / numMappers * numKVPairs).toInt var arr1 = new Array[(Int, Array[Byte])](numKVPairs) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala index aa530a7121bd0..09d3fe91e42c8 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala @@ -101,7 +101,7 @@ private[sink] trait Logging { private def initializeLogging() { Logging.initialized = true - // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads + // Force a call into slf4j to initialize it. Avoids this happening from multiple threads // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html log } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala index b9d4e762ca05d..3555fa68b67a1 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala @@ -77,7 +77,7 @@ private[flume] class FlumeBatchFetcher(receiver: FlumePollingReceiver) extends R /** * Gets a batch of events from the specified client. This method does not handle any exceptions - * which will be propogated to the caller. + * which will be propagated to the caller. * @param client Client to get events from * @return [[Some]] which contains the event batch if Flume sent any events back, else [[None]] */ @@ -96,8 +96,8 @@ private[flume] class FlumeBatchFetcher(receiver: FlumePollingReceiver) extends R } /** - * Store the events in the buffer to Spark. This method will not propogate any exceptions, - * but will propogate any other errors. + * Store the events in the buffer to Spark. This method will not propagate any exceptions, + * but will propagate any other errors. * @param buffer The buffer to store * @return true if the data was stored without any exception being thrown, else false */ diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala index 8a66621a3125c..726b5d8ec3d3b 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala @@ -167,7 +167,7 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = { getLeaderOffsets(topicAndPartitions, before, 1).right.map { r => r.map { kv => - // mapValues isnt serializable, see SI-7005 + // mapValues isn't serializable, see SI-7005 kv._1 -> kv._2.head } } diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala index b5b76cb92d866..23b74da64237a 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala @@ -132,7 +132,7 @@ private[kinesis] object KinesisRecordProcessor extends Logging { * Retry the given amount of times with a random backoff time (millis) less than the * given maxBackOffMillis * - * @param expression expression to evalute + * @param expression expression to evaluate * @param numRetriesLeft number of retries left * @param maxBackOffMillis: max millis between retries * diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala index 2555332d222da..905c33834df16 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDDSuite.scala @@ -122,7 +122,7 @@ abstract class KinesisBackedBlockRDDTests(aggregateTestData: Boolean) testIsBlockValid = true) } - testIfEnabled("Test whether RDD is valid after removing blocks from block anager") { + testIfEnabled("Test whether RDD is valid after removing blocks from block manager") { testRDD(numPartitions = 2, numPartitionsInBM = 2, numPartitionsInKinesis = 2, testBlockRemove = true) } diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala index fd15b6ccdc889..deac9090e2f48 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala @@ -194,7 +194,7 @@ class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAft verify(checkpointerMock, times(1)).checkpoint() } - test("retry failed after exhausing all retries") { + test("retry failed after exhausting all retries") { val expectedErrorMessage = "final try error message" when(checkpointerMock.checkpoint()) .thenThrow(new ThrottlingException("error message")) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index fe884d0022500..5485e30f5a2c9 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -297,7 +297,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab /** * Restricts the graph to only the vertices and edges satisfying the predicates. The resulting - * subgraph satisifies + * subgraph satisfies * * {{{ * V' = {v : for all v in V where vpred(v)} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index d537b6141cc90..fcb1b5999fae7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -236,11 +236,11 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * @param preprocess a function to compute new vertex and edge data before filtering * @param epred edge pred to filter on after preprocess, see more details under * [[org.apache.spark.graphx.Graph#subgraph]] - * @param vpred vertex pred to filter on after prerocess, see more details under + * @param vpred vertex pred to filter on after preprocess, see more details under * [[org.apache.spark.graphx.Graph#subgraph]] * @tparam VD2 vertex type the vpred operates on * @tparam ED2 edge type the epred operates on - * @return a subgraph of the orginal graph, with its data unchanged + * @return a subgraph of the original graph, with its data unchanged * * @example This function can be used to filter the graph based on some property, without * changing the vertex and edge values in your program. For example, we could remove the vertices diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index 6dab465fb9012..a4e293d74a012 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -49,7 +49,7 @@ object ShippableVertexPartition { /** * Construct a `ShippableVertexPartition` from the given vertices with the specified routing * table, filling in missing vertices mentioned in the routing table using `defaultVal`, - * and merging duplicate vertex atrribute with mergeFunc. + * and merging duplicate vertex attribute with mergeFunc. */ def apply[VD: ClassTag]( iter: Iterator[(VertexId, VD)], routingTable: RoutingTablePartition, defaultVal: VD, diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index 46faad2e68c50..00ba358a9b4a6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -54,7 +54,7 @@ import org.apache.spark.graphx._ * }}} * * `alpha` is the random reset probability (typically 0.15), `inNbrs[i]` is the set of - * neighbors whick link to `i` and `outDeg[j]` is the out degree of vertex `j`. + * neighbors which link to `i` and `outDeg[j]` is the out degree of vertex `j`. * * Note that this is not the "normalized" PageRank and as a consequence pages that have no * inlinks will have a PageRank of alpha. @@ -209,7 +209,7 @@ object PageRank extends Logging { } // Set the weight on the edges based on the degree .mapTriplets( e => 1.0 / e.srcAttr ) - // Set the vertex attributes to (initalPR, delta = 0) + // Set the vertex attributes to (initialPR, delta = 0) .mapVertices { (id, attr) => if (id == src) (resetProb, Double.NegativeInfinity) else (0.0, 0.0) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index 6aab28ff05355..dde25b96594be 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -30,7 +30,7 @@ package object graphx { */ type VertexId = Long - /** Integer identifer of a graph partition. Must be less than 2^30. */ + /** Integer identifier of a graph partition. Must be less than 2^30. */ // TODO: Consider using Char. type PartitionID = Int diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala index a6d0cb6409664..d76e84ed8c9ed 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala @@ -92,7 +92,7 @@ private[graphx] object BytecodeUtils { /** * Given the class name, return whether we should look into the class or not. This is used to - * skip examing a large quantity of Java or Scala classes that we know for sure wouldn't access + * skip examining a large quantity of Java or Scala classes that we know for sure wouldn't access * the closures. Note that the class name is expected in ASM style (i.e. use "/" instead of "."). */ private def skipClass(className: String): Boolean = { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index f497e001dfa4f..cb981797d3239 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -229,7 +229,7 @@ class GraphSuite extends SparkFunSuite with LocalSparkContext { test("subgraph") { withSpark { sc => - // Create a star graph of 10 veritces. + // Create a star graph of 10 vertices. val n = 10 val star = starGraph(sc, n) // Take only vertices whose vids are even diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala index f21b623e93253..2cd94fa8f5856 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala @@ -523,7 +523,7 @@ private[ml] object FeedForwardTopology { /** * Creates a multi-layer perceptron * @param layerSizes sizes of layers including input and output size - * @param softmax wether to use SoftMax or Sigmoid function for an output layer. + * @param softmax whether to use SoftMax or Sigmoid function for an output layer. * Softmax is default * @return multilayer perceptron topology */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala index 521d209a8f0ed..27554acdf3c26 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala @@ -481,7 +481,7 @@ object NominalAttribute extends AttributeFactory { * A binary attribute. * @param name optional name * @param index optional index - * @param values optionla values. If set, its size must be 2. + * @param values optional values. If set, its size must be 2. */ @DeveloperApi class BinaryAttribute private[ml] ( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala index f8de4e2220c4d..c8ec0c16851f1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala @@ -83,7 +83,7 @@ class RankingMetrics[T: ClassTag](predictionAndLabels: RDD[(Array[T], Array[T])] /** * Returns the mean average precision (MAP) of all the queries. * If a query has an empty ground truth set, the average precision will be zero and a log - * warining is generated. + * warning is generated. */ lazy val meanAveragePrecision: Double = { predictionAndLabels.map { case (pred, lab) => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index 07eb750b06a3b..790d6b101ee5f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -58,7 +58,7 @@ class AssociationRules private[fpm] ( /** * Computes the association rules with confidence above [[minConfidence]]. * @param freqItemsets frequent itemset model obtained from [[FPGrowth]] - * @return a [[Set[Rule[Item]]] containing the assocation rules. + * @return a [[Set[Rule[Item]]] containing the association rules. * */ @Since("1.5.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 240781bcd335b..58fd010e4905f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -158,7 +158,7 @@ object LinearDataGenerator { /** * Generate an RDD containing sample data for Linear Regression models - including Ridge, Lasso, - * and uregularized variants. + * and unregularized variants. * * @param sc SparkContext to be used for generating the RDD. * @param nexamples Number of examples that will be contained in the RDD. diff --git a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala index 71f4926290b20..114a238462a3d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/source/libsvm/LibSVMRelationSuite.scala @@ -88,7 +88,7 @@ class LibSVMRelationSuite extends SparkFunSuite with MLlibTestSparkContext { val df = sqlContext.read.format("libsvm").load(path) val tempDir2 = Utils.createTempDir() val writepath = tempDir2.toURI.toString - // TODO: Remove requirement to coalesce by supporting mutiple reads. + // TODO: Remove requirement to coalesce by supporting multiple reads. df.coalesce(1).write.format("libsvm").mode(SaveMode.Overwrite).save(writepath) val df2 = sqlContext.read.format("libsvm").load(writepath) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index cea0adc55c076..28fada7053d65 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -496,7 +496,7 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext w * features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5)) * weights = coef(glmnet(features,label, family="multinomial", alpha = 0, lambda = 0)) * - * The model weights of mutinomial logstic regression in R have `K` set of linear predictors + * The model weights of multinomial logistic regression in R have `K` set of linear predictors * for `K` classes classification problem; however, only `K-1` set is required if the first * outcome is chosen as a "pivot", and the other `K-1` outcomes are separately regressed against * the pivot outcome. This can be done by subtracting the first weights from those `K-1` set diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala index 142b90e764a7c..46fcebe132749 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala @@ -144,7 +144,7 @@ class HypothesisTestSuite extends SparkFunSuite with MLlibTestSparkContext { assert(chi.size === numCols) assert(chi(1000) != null) // SPARK-3087 - // Detect continous features or labels + // Detect continuous features or labels val random = new Random(11L) val continuousLabel = Seq.fill(100000)(LabeledPoint(random.nextDouble(), Vectors.dense(random.nextInt(2)))) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index dca8ea815aa6a..5518bdf527c8a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -1075,7 +1075,7 @@ object DecisionTreeSuite extends SparkFunSuite { assert(a.isLeaf === b.isLeaf) assert(a.split === b.split) (a.stats, b.stats) match { - // TODO: Check other fields besides the infomation gain. + // TODO: Check other fields besides the information gain. case (Some(aStats), Some(bStats)) => assert(aStats.gain === bStats.gain) case (None, None) => case _ => throw new AssertionError( diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index c4c8d8870f8b9..faa52bf18cd2d 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -334,7 +334,7 @@ object MimaExcludes { excludePackage("org.apache.spark.sql.columnar"), // The shuffle package is considered private. excludePackage("org.apache.spark.shuffle"), - // The collections utlities are considered pricate. + // The collections utilities are considered private. excludePackage("org.apache.spark.util.collection") ) ++ MimaBuild.excludeSparkClass("streaming.flume.FlumeTestUtils") ++ @@ -639,7 +639,7 @@ object MimaExcludes { Seq( MimaBuild.excludeSparkPackage("deploy"), MimaBuild.excludeSparkPackage("ml"), - // SPARK-7910 Adding a method to get the partioner to JavaRDD, + // SPARK-7910 Adding a method to get the partitioner to JavaRDD, ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitioner"), // SPARK-5922 Adding a generalized diff(other: RDD[(VertexId, VD)]) to VertexRDD ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.graphx.VertexRDD.diff"), @@ -657,7 +657,7 @@ object MimaExcludes { ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.scheduler.OutputCommitCoordinator$OutputCommitCoordinatorEndpoint") ) ++ Seq( - // SPARK-4655 - Making Stage an Abstract class broke binary compatility even though + // SPARK-4655 - Making Stage an Abstract class broke binary compatibility even though // the stage class is defined as private[spark] ProblemFilters.exclude[AbstractClassProblem]("org.apache.spark.scheduler.Stage") ) ++ Seq( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index f108264861eee..1219d4d453e13 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -305,7 +305,7 @@ trait Row extends Serializable { * @throws ClassCastException when data type does not match. */ def getStruct(i: Int): Row = { - // Product and Row both are recoginized as StructType in a Row + // Product and Row both are recognized as StructType in a Row val t = get(i) if (t.isInstanceOf[Product]) { Row.fromTuple(t.asInstanceOf[Product]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index c12b5c20ea7bf..bf07f4557a5b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -29,7 +29,7 @@ import org.apache.spark.util.Utils */ object ScalaReflection extends ScalaReflection { val universe: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe - // Since we are creating a runtime mirror usign the class loader of current thread, + // Since we are creating a runtime mirror using the class loader of current thread, // we need to use def at here. So, every time we call mirror, it is using the // class loader of the current thread. // SPARK-13640: Synchronize this because universe.runtimeMirror is not thread-safe in Scala 2.10. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala index ad56c9864979b..9c38dd2ee4e53 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala @@ -155,7 +155,7 @@ object DecimalPrecision extends Rule[LogicalPlan] { * * Note that technically this is an "optimization" and should go into the optimizer. However, * by the time the optimizer runs, these comparison expressions would be pretty hard to pattern - * match because there are multuple (at least 2) levels of casts involved. + * match because there are multiple (at least 2) levels of casts involved. * * There are a lot more possible rules we can implement, but we don't do them * because we are not sure how common they are. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala index 38c1641f73d9f..2e30d83a60970 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DistinctAggregationRewriter.scala @@ -96,7 +96,7 @@ import org.apache.spark.sql.types.IntegerType * This rule duplicates the input data by two or more times (# distinct groups + an optional * non-distinct group). This will put quite a bit of memory pressure of the used aggregate and * exchange operators. Keeping the number of distinct groups as low a possible should be priority, - * we could improve this in the current rule by applying more advanced expression cannocalization + * we could improve this in the current rule by applying more advanced expression canonicalization * techniques. */ object DistinctAggregationRewriter extends Rule[LogicalPlan] { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 57bdb164e1a0d..0f85f44ffa768 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -307,7 +307,7 @@ object HiveTypeCoercion { case p @ Equality(left @ TimestampType(), right @ StringType()) => p.makeCopy(Array(left, Cast(right, TimestampType))) - // We should cast all relative timestamp/date/string comparison into string comparisions + // We should cast all relative timestamp/date/string comparison into string comparisons // This behaves as a user would expect because timestamp strings sort lexicographically. // i.e. TimeStamp(2013-01-01 00:00 ...) < "2014" = true case p @ BinaryComparison(left @ StringType(), right @ DateType()) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala index 3831535574205..8bdf9b29c9641 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -53,7 +53,7 @@ object AttributeSet { * cosmetically (e.g., the names have different capitalizations). * * Note that we do not override equality for Attribute references as it is really weird when - * `AttributeReference("a"...) == AttrributeReference("b", ...)`. This tactic leads to broken tests, + * `AttributeReference("a"...) == AttributeReference("b", ...)`. This tactic leads to broken tests, * and also makes doing transformations hard (we always try keep older trees instead of new ones * when the transformation was a no-op). */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala index acea049adca3d..644a5b28a2151 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSet.scala @@ -36,7 +36,7 @@ object ExpressionSet { * Internally this set uses the canonical representation, but keeps also track of the original * expressions to ease debugging. Since different expressions can share the same canonical * representation, this means that operations that extract expressions from this set are only - * guranteed to see at least one such expression. For example: + * guaranteed to see at least one such expression. For example: * * {{{ * val set = AttributeSet(a + 1, 1 + a) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 22184f1ddfbb5..500ff447a9754 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -973,7 +973,7 @@ case class ScalaUDF( // scalastyle:on line.size.limit - // Generate codes used to convert the arguments to Scala type for user-defined funtions + // Generate codes used to convert the arguments to Scala type for user-defined functions private[this] def genCodeForConverter(ctx: CodegenContext, index: Int): String = { val converterClassName = classOf[Any => Any].getName val typeConvertersClassName = CatalystTypeConverters.getClass.getName + ".MODULE$" 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 c4265a753933f..3dbe6348986b2 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 @@ -126,7 +126,7 @@ class CodegenContext { * For expressions that appear more than once, generate additional code to prevent * recomputing the value. * - * For example, consider two exprsesion generated from this SQL statement: + * For example, consider two expression generated from this SQL statement: * SELECT (col1 + col2), (col1 + col2) / col3. * * equivalentExpressions will match the tree containing `col1 + col2` and it will only @@ -140,7 +140,7 @@ class CodegenContext { // Foreach expression that is participating in subexpression elimination, the state to use. val subExprEliminationExprs = mutable.HashMap.empty[Expression, SubExprEliminationState] - // The collection of sub-exression result resetting methods that need to be called on each row. + // The collection of sub-expression result resetting methods that need to be called on each row. val subexprFunctions = mutable.ArrayBuffer.empty[String] def declareAddedFunctions(): String = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 5ceb36513f840..103ab365e3190 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -214,7 +214,7 @@ case class CaseWhen(branches: Seq[(Expression, Expression)], elseValue: Option[E /** Factory methods for CaseWhen. */ object CaseWhen { - // The maxium number of switches supported with codegen. + // The maximum number of switches supported with codegen. val MAX_NUM_CASES_FOR_CODEGEN = 20 def apply(branches: Seq[(Expression, Expression)], elseValue: Expression): CaseWhen = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala index b95c5dd892d06..7eba617fcde59 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects.scala @@ -364,7 +364,7 @@ object MapObjects { * used as input for the `lambdaFunction`. It also carries the element type info. * @param lambdaFunction A function that take the `loopVar` as input, and used as lambda function * to handle collection elements. - * @param inputData An expression that when evaluted returns a collection object. + * @param inputData An expression that when evaluated returns a collection object. */ case class MapObjects private( loopVar: LambdaVariable, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala index e4417e0955143..da90ddbd63afb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala @@ -66,7 +66,7 @@ object NumberConverter { * negative digit is found, ignore the suffix starting there. * * @param radix must be between MIN_RADIX and MAX_RADIX - * @param fromPos is the first element that should be conisdered + * @param fromPos is the first element that should be considered * @return the result should be treated as an unsigned 64-bit integer. */ private def encode(radix: Int, fromPos: Int): Long = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index de9a56dc9c064..4e7bbc38d60ea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -276,7 +276,7 @@ class AnalysisErrorSuite extends AnalysisTest { test("SPARK-6452 regression test") { // CheckAnalysis should throw AnalysisException when Aggregate contains missing attribute(s) - // Since we manually construct the logical plan at here and Sum only accetp + // Since we manually construct the logical plan at here and Sum only accept // LongType, DoubleType, and DecimalType. We use LongType as the type of a. val plan = Aggregate( 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 aa1d2b08613dd..8b568b6dd6acd 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 @@ -250,7 +250,7 @@ class AnalysisSuite extends AnalysisTest { assertAnalysisSuccess(plan) } - test("SPARK-8654: different types in inlist but can be converted to a commmon type") { + test("SPARK-8654: different types in inlist but can be converted to a common type") { val plan = Project(Alias(In(Literal(null), Seq(Literal(1), Literal(1.2345))), "a")() :: Nil, LocalRelation() ) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index c30434a0063b0..6f289dcc475cd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -205,7 +205,7 @@ class HiveTypeCoercionSuite extends PlanTest { Project(Seq(Alias(transformed, "a")()), testRelation)) } - test("cast NullType for expresions that implement ExpectsInputTypes") { + test("cast NullType for expressions that implement ExpectsInputTypes") { import HiveTypeCoercionSuite._ ruleTest(HiveTypeCoercion.ImplicitTypeCasts, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala index ce42e5784ccd2..0b350c6a98255 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionSetSuite.scala @@ -70,7 +70,7 @@ class ExpressionSetSuite extends SparkFunSuite { // Not commutative setTest(2, aUpper - bUpper, bUpper - aUpper) - // Reversable + // Reversible setTest(1, aUpper > bUpper, bUpper < aUpper) setTest(1, aUpper >= bUpper, bUpper <= aUpper) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala index 0dbfb01e881f5..f5374229ca5cd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlusSuite.scala @@ -131,7 +131,7 @@ class HyperLogLogPlusPlusSuite extends SparkFunSuite { i += 1 } - // Merge the lower and upper halfs. + // Merge the lower and upper halves. hll.merge(buffer1a, buffer1b) // Create the other buffer in reverse 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 f7ba61d2b804f..1751720a7db88 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 @@ -78,7 +78,7 @@ class TypedColumn[-T, U]( * * {{{ * df("columnName") // On a specific DataFrame. - * col("columnName") // A generic column no yet associcated with a DataFrame. + * col("columnName") // A generic column no yet associated with a DataFrame. * col("columnName.field") // Extracting a struct field * col("`a.column.with.dots`") // Escape `.` in column names. * $"columnName" // Scala short hand for a named column. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala index 472ae716f1530..a8700de135ce4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedDataset.scala @@ -223,7 +223,7 @@ class GroupedDataset[K, V] private[sql]( * Internal helper function for building typed aggregations that return tuples. For simplicity * and code reuse, we do this without the help of the type system and then use helper functions * that cast appropriately for the user facing interface. - * TODO: does not handle aggrecations that return nonflat results, + * TODO: does not handle aggregations that return nonflat results, */ protected def aggUntyped(columns: TypedColumn[_, _]*): Dataset[_] = { val encoders = columns.map(_.encoder) 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 3be4cce045fea..a92c99e06ff43 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 @@ -52,7 +52,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ protected def sparkContext = sqlContext.sparkContext // sqlContext will be null when we are being deserialized on the slaves. In this instance - // the value of subexpressionEliminationEnabled will be set by the desserializer after the + // the value of subexpressionEliminationEnabled will be set by the deserializer after the // constructor has run. val subexpressionEliminationEnabled: Boolean = if (sqlContext != null) { sqlContext.conf.subexpressionEliminationEnabled @@ -65,7 +65,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ */ private val prepareCalled = new AtomicBoolean(false) - /** Overridden make copy also propogates sqlContext to copied plan. */ + /** Overridden make copy also propagates sqlContext to copied plan. */ override def makeCopy(newArgs: Array[AnyRef]): SparkPlan = { SQLContext.setActive(sqlContext) super.makeCopy(newArgs) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala index 3ec01185c4328..f9d606e37ea89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnType.scala @@ -40,7 +40,7 @@ import org.apache.spark.unsafe.types.UTF8String * so we do not have helper methods for them. * * - * WARNNING: This only works with HeapByteBuffer + * WARNING: This only works with HeapByteBuffer */ private[columnar] object ByteBufferHelper { def getInt(buffer: ByteBuffer): Int = { 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 c3f8d7f75a23a..18a460fc85ed4 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 @@ -88,7 +88,7 @@ private[sql] object PartitioningUtils { }.unzip // We create pairs of (path -> path's partition value) here - // If the corresponding partition value is None, the pair will be skiped + // If the corresponding partition value is None, the pair will be skipped val pathsWithPartitionValues = paths.zip(partitionValues).flatMap(x => x._2.map(x._1 -> _)) if (pathsWithPartitionValues.isEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index e295722cacf15..64a820c6d741f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -70,7 +70,7 @@ object JdbcUtils extends Logging { // Somewhat hacky, but there isn't a good way to identify whether a table exists for all // SQL database systems using JDBC meta data calls, considering "table" could also include - // the database name. Query used to find table exists can be overriden by the dialects. + // the database name. Query used to find table exists can be overridden by the dialects. Try { val statement = conn.prepareStatement(dialect.getTableExistsQuery(table)) try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala index c65a7bcff8503..79e4491026b65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchPythonEvaluation.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.types.{StructField, StructType} /** - * A physical plan that evalutes a [[PythonUDF]], one partition of tuples at a time. + * A physical plan that evaluates a [[PythonUDF]], one partition of tuples at a time. * * Python evaluation works by sending the necessary (projected) input data via a socket to an * external Python process, and combine the result from the Python process with the original row. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala index 1bd71b6b02ea9..e3b2d2f67ee0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala @@ -38,7 +38,7 @@ trait Sink { * Accepts a new batch of data as well as a [[Offset]] that denotes how far in the input * data computation has progressed to. When computation restarts after a failure, it is important * that a [[Sink]] returns the same [[Offset]] as the most recent batch of data that - * has been persisted durrably. Note that this does not necessarily have to be the + * has been persisted durably. Note that this does not necessarily have to be the * [[Offset]] for the most recent batch of data that was given to the sink. For example, * it is valid to buffer data before persisting, as long as the [[Offset]] is stored * transactionally as data is eventually persisted. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index 096477ce0e511..d7ff44afadf22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -100,7 +100,7 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) /** * A sink that stores the results in memory. This [[Sink]] is primarily intended for use in unit - * tests and does not provide durablility. + * tests and does not provide durability. */ class MemorySink(schema: StructType) extends Sink with Logging { /** An order list of batches that have been written to this [[Sink]]. */ 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 384102e5eaa5b..59429d254ebb6 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 @@ -441,7 +441,7 @@ object SQLConf { // NOTE: // // 1. Instead of SQLConf, this option *must be set in Hadoop Configuration*. - // 2. This option can be overriden by "spark.sql.parquet.output.committer.class". + // 2. This option can be overridden by "spark.sql.parquet.output.committer.class". val OUTPUT_COMMITTER_CLASS = stringConf("spark.sql.sources.outputCommitterClass", isPublic = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index e865dbe6b5063..a7a826bc7a8d6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -80,7 +80,7 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { // Verify that the splits span the entire dataset assert(splits.flatMap(_.collect()).toSet == data.collect().toSet) - // Verify that the splits don't overalap + // Verify that the splits don't overlap assert(splits(0).intersect(splits(1)).collect().isEmpty) // Verify that the results are deterministic across multiple runs 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 9f32c8bf95ad6..d7fa23651bcee 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 @@ -46,7 +46,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { 1, 1, 1) } - test("SPARK-12404: Datatype Helper Serializablity") { + test("SPARK-12404: Datatype Helper Serializability") { val ds = sparkContext.parallelize(( new Timestamp(0), new Date(0), 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 182f287dd001c..98d0008489f4d 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 @@ -986,7 +986,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("SET commands with illegal or inappropriate argument") { sqlContext.conf.clear() - // Set negative mapred.reduce.tasks for automatically determing + // Set negative mapred.reduce.tasks for automatically determining // the number of reducers is not supported intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-1")) intercept[IllegalArgumentException](sql(s"SET mapred.reduce.tasks=-01")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala index 7a5b63911546f..81078dc6a0450 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StreamTest.scala @@ -80,7 +80,7 @@ trait StreamTest extends QueryTest with Timeouts { trait StreamMustBeRunning /** - * Adds the given data to the stream. Subsuquent check answers will block until this data has + * Adds the given data to the stream. Subsequent check answers will block until this data has * been processed. */ object AddData { @@ -109,7 +109,7 @@ trait StreamTest extends QueryTest with Timeouts { /** * Checks to make sure that the current data stored in the sink matches the `expectedAnswer`. - * This operation automatically blocks untill all added data has been processed. + * This operation automatically blocks until all added data has been processed. */ object CheckAnswer { def apply[A : Encoder](data: A*): CheckAnswerRows = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala index 7af3f94aefea2..3a7cb25b4fa9b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala @@ -80,7 +80,7 @@ class InferSchemaSuite extends SparkFunSuite { assert(CSVInferSchema.inferField(BooleanType, "\\N", "\\N") == BooleanType) } - test("Merging Nulltypes should yeild Nulltype.") { + test("Merging Nulltypes should yield Nulltype.") { val mergedNullTypes = CSVInferSchema.mergeRowTypes(Array(NullType), Array(NullType)) assert(mergedNullTypes.deep == Array(NullType).deep) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 1ef517324d7cb..f66deea06589c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -359,7 +359,7 @@ class JDBCSuite extends SparkFunSuite .collect().length === 3) } - test("Partioning on column that might have null values.") { + test("Partitioning on column that might have null values.") { assert( sqlContext.read.jdbc(urlWithUserAndPass, "TEST.EMP", "theid", 0, 4, 3, new Properties) .collect().length === 4) @@ -372,7 +372,7 @@ class JDBCSuite extends SparkFunSuite .collect().length === 4) } - test("SELECT * on partitioned table with a nullable partioncolumn") { + test("SELECT * on partitioned table with a nullable partition column") { assert(sql("SELECT * FROM nullparts").collect().size == 4) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 26c1ff520406c..99f1661ad0d15 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -339,7 +339,7 @@ class TableScanSuite extends DataSourceTest with SharedSQLContext { test("exceptions") { // Make sure we do throw correct exception when users use a relation provider that - // only implements the RelationProvier or the SchemaRelationProvider. + // only implements the RelationProvider or the SchemaRelationProvider. val schemaNotAllowed = intercept[Exception] { sql( """ diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 54fffb971dbc3..694bd97515b86 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -68,7 +68,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { * with one of these strings is found, fail the test immediately. * The default value is `Seq("Error:")` * - * @param queriesAndExpectedAnswers one or more tupes of query + answer + * @param queriesAndExpectedAnswers one or more tuples of query + answer */ def runCliWithin( timeout: FiniteDuration, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 56acb87c800d3..739fbaf4446ff 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -245,7 +245,7 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging val tableName = tableNameParts.map { case Token(p, Nil) => p }.mkString(".") DropTable(tableName, ifExists.nonEmpty) - // Support "ANALYZE TABLE tableNmae COMPUTE STATISTICS noscan" + // Support "ANALYZE TABLE tableName COMPUTE STATISTICS noscan" case Token("TOK_ANALYZE", Token("TOK_TAB", Token("TOK_TABNAME", tableNameParts) :: partitionSpec) :: isNoscan) => // Reference: @@ -535,7 +535,7 @@ private[hive] class HiveQl(conf: ParserConf) extends SparkQl(conf) with Logging case Token("TOK_STORAGEHANDLER", _) => throw new AnalysisException( "CREATE TABLE AS SELECT cannot be used for a non-native table") - case _ => // Unsupport features + case _ => // Unsupported features } CreateTableAsSelect(tableDesc, nodeToPlan(query), allowExisting.isDefined) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala index 059ad8b1f7274..8240f2f2220cd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala @@ -89,7 +89,7 @@ private[orc] object OrcFileOperator extends Logging { } def listOrcFiles(pathStr: String, conf: Configuration): Seq[Path] = { - // TODO: Check if the paths comming in are already qualified and simplify. + // TODO: Check if the paths coming in are already qualified and simplify. val origPath = new Path(pathStr) val fs = origPath.getFileSystem(conf) val path = origPath.makeQualified(fs.getUri, fs.getWorkingDirectory) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 1053246fc2958..5e452d107dc75 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -143,7 +143,7 @@ abstract class HiveComparisonTest 0D } - s"""SQLBuiler statistics: + s"""SQLBuilder statistics: |- Total query number: $numTotalQueries |- Number of convertible queries: $numConvertibleQueries |- Percentage of convertible queries: $percentage% diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 10024874472f2..d905f0cd68a4a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -602,7 +602,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |select * where key = 4 """.stripMargin) - // test get_json_object again Hive, because the HiveCompatabilitySuite cannot handle result + // test get_json_object again Hive, because the HiveCompatibilitySuite cannot handle result // with newline in it. createQueryTest("get_json_object #1", "SELECT get_json_object(src_json.json, '$') FROM src_json") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 11a4c7dfd011f..16c575bcc13ab 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -205,7 +205,7 @@ class CheckpointWriter( // also use the latest checkpoint time as the file name, so that we can recovery from the // latest checkpoint file. // - // Note: there is only one thread writting the checkpoint files, so we don't need to worry + // Note: there is only one thread writing the checkpoint files, so we don't need to worry // about thread-safety. val checkpointFile = Checkpoint.checkpointFile(checkpointDir, latestCheckpointTime) val backupFile = Checkpoint.checkpointBackupFile(checkpointDir, latestCheckpointTime) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 25e61578a1860..e7f3a213d468e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -299,7 +299,7 @@ class StreamingContext private[streaming] ( /** * Create a input stream from TCP source hostname:port. Data is received using - * a TCP socket and the receive bytes it interepreted as object using the given + * a TCP socket and the receive bytes it interpreted as object using the given * converter. * @param hostname Hostname to connect to for receiving data * @param port Port to connect to for receiving data diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index aad9a12c15246..2a80cf4466588 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -155,7 +155,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control - * thepartitioning of each RDD. + * the partitioning of each RDD. */ def reduceByKey(func: JFunction2[V, V, V], partitioner: Partitioner): JavaPairDStream[K, V] = { dstream.reduceByKey(func, partitioner) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 860b8027253fd..05f4da6face4d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -530,7 +530,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * Return the current state of the context. The context can be in three possible states - *
    *
  • - * StreamingContextState.INTIALIZED - The context has been created, but not been started yet. + * StreamingContextState.INITIALIZED - The context has been created, but not been started yet. * Input DStreams, transformations and output operations can be created on the context. *
  • *
  • diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 1dcdb64e289bd..d6ff96e1fc696 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -446,7 +446,7 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) * remember the partitioner despite the key being changed. * @param partitioner Partitioner for controlling the partitioning of each RDD in the new * DStream - * @param rememberPartitioner Whether to remember the paritioner object in the generated RDDs. + * @param rememberPartitioner Whether to remember the partitioner object in the generated RDDs. * @tparam S State type */ def updateStateByKey[S: ClassTag]( @@ -490,7 +490,7 @@ class PairDStreamFunctions[K, V](self: DStream[(K, V)]) * remember the partitioner despite the key being changed. * @param partitioner Partitioner for controlling the partitioning of each RDD in the new * DStream - * @param rememberPartitioner Whether to remember the paritioner object in the generated RDDs. + * @param rememberPartitioner Whether to remember the partitioner object in the generated RDDs. * @param initialRDD initial state value of each key. * @tparam S State type */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala index 080bc873fa0a8..47eb9b806fa7d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/TransformedDStream.scala @@ -55,7 +55,7 @@ class TransformedDStream[U: ClassTag] ( /** * Wrap a body of code such that the call site and operation scope * information are passed to the RDDs created in this body properly. - * This has been overriden to make sure that `displayInnerRDDOps` is always `true`, that is, + * This has been overridden to make sure that `displayInnerRDDOps` is always `true`, that is, * the inner scopes and callsites of RDDs generated in `DStream.transform` are always * displayed in the UI. */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index 430f35a400dbe..d6fcc582b9c4d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -257,7 +257,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } batchUIData.foreach { _batchUIData => // We use an Iterable rather than explicitly converting to a seq so that updates - // will propegate + // will propagate val outputOpIdToSparkJobIds: Iterable[OutputOpIdAndSparkJobId] = Option(batchTimeToOutputOpIdSparkJobIdPair.get(batchTime).asScala) .getOrElse(Seq.empty) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala index 2be1d6df86f89..3a21cfae5ac2f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala @@ -177,7 +177,7 @@ private[streaming] class OpenHashMapBasedStateMap[K, S]( new OpenHashMapBasedStateMap[K, S](this, deltaChainThreshold = deltaChainThreshold) } - /** Whether the delta chain lenght is long enough that it should be compacted */ + /** Whether the delta chain length is long enough that it should be compacted */ def shouldCompact: Boolean = { deltaChainLength >= deltaChainThreshold } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala index 403400904bac2..3b662ec1833aa 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MapWithStateSuite.scala @@ -518,7 +518,7 @@ class MapWithStateSuite extends SparkFunSuite val mapWithStateStream = dstream.map { _ -> 1 }.mapWithState( StateSpec.function(runningCount)) - // Set internval make sure there is one RDD checkpointing + // Set interval make sure there is one RDD checkpointing mapWithStateStream.checkpoint(checkpointDuration) mapWithStateStream.stateSnapshots() } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 45424f9bac05a..95c1609d8e9a0 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -202,13 +202,13 @@ class ReceivedBlockHandlerSuite blockManager = createBlockManager(12000, sparkConf) // there is not enough space to store this block in MEMORY, - // But BlockManager will be able to sereliaze this block to WAL + // But BlockManager will be able to serialize this block to WAL // and hence count returns correct value. testRecordcount(false, StorageLevel.MEMORY_ONLY, IteratorBlock((List.fill(70)(new Array[Byte](100))).iterator), blockManager, Some(70)) // there is not enough space to store this block in MEMORY, - // But BlockManager will be able to sereliaze this block to DISK + // But BlockManager will be able to serialize this block to DISK // and hence count returns correct value. testRecordcount(true, StorageLevel.MEMORY_AND_DISK, IteratorBlock((List.fill(70)(new Array[Byte](100))).iterator), blockManager, Some(70)) @@ -272,7 +272,7 @@ class ReceivedBlockHandlerSuite } /** - * Test storing of data using different types of Handler, StorageLevle and ReceivedBlocks + * Test storing of data using different types of Handler, StorageLevel and ReceivedBlocks * and verify the correct record count */ private def testRecordcount(isBlockManagedBasedBlockHandler: Boolean, diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala index a4871b460eb4d..6763ac64da287 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala @@ -97,7 +97,7 @@ class ReceiverInputDStreamSuite extends TestSuiteBase with BeforeAndAfterAll { assert(blockRDD.walRecordHandles.toSeq === blockInfos.map { _.walRecordHandleOption.get }) } - testWithWAL("createBlockRDD creates BlockRDD when some block info dont have WAL info") { + testWithWAL("createBlockRDD creates BlockRDD when some block info don't have WAL info") { receiverStream => val blockInfos1 = Seq.fill(2) { createBlockInfo(withWALInfo = true) } val blockInfos2 = Seq.fill(3) { createBlockInfo(withWALInfo = false) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala index 7a76cafc9a11c..484f3733e8423 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala @@ -182,7 +182,7 @@ class StateMapSuite extends SparkFunSuite { * * - These operations are done on a test map in "sets". After each set, the map is "copied" * to create a new map, and the next set of operations are done on the new one. This tests - * whether the map data persistes correctly across copies. + * whether the map data persist correctly across copies. * * - Within each set, there are a number of operations to test whether the map correctly * updates and removes data without affecting the parent state map. diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 197b3d143995a..2159edce2bf52 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -147,7 +147,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with Timeo } } - test("start with non-seriazable DStream checkpoints") { + test("start with non-serializable DStream checkpoints") { val checkpointDir = Utils.createTempDir() ssc = new StreamingContext(conf, batchDuration) ssc.checkpoint(checkpointDir.getAbsolutePath) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 82cd63bcafc97..8269963edffa8 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -56,7 +56,7 @@ private[streaming] class DummyInputDStream(ssc: StreamingContext) extends InputD /** * This is a input stream just for the testsuites. This is equivalent to a checkpointable, * replayable, reliable message queue like Kafka. It requires a sequence as input, and - * returns the i_th element at the i_th batch unde manual clock. + * returns the i_th element at the i_th batch under manual clock. */ class TestInputStream[T: ClassTag](_ssc: StreamingContext, input: Seq[Seq[T]], numPartitions: Int) extends InputDStream[T](_ssc) { diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala index 6e95bb97105fd..498471b23b51e 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/AMDelegationTokenRenewer.scala @@ -115,7 +115,7 @@ private[yarn] class AMDelegationTokenRenewer( } } // Schedule update of credentials. This handles the case of updating the tokens right now - // as well, since the renenwal interval will be 0, and the thread will get scheduled + // as well, since the renewal interval will be 0, and the thread will get scheduled // immediately. scheduleRenewal(driverTokenRenewerRunnable) } diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 9f91d182ebc32..9cdbd6da62185 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -186,9 +186,9 @@ private[yarn] class ExecutorRunnable( else { // If no java_opts specified, default to using -XX:+CMSIncrementalMode // It might be possible that other modes/config is being done in - // spark.executor.extraJavaOptions, so we dont want to mess with it. - // In our expts, using (default) throughput collector has severe perf ramnifications in - // multi-tennent machines + // spark.executor.extraJavaOptions, so we don't want to mess with it. + // In our expts, using (default) throughput collector has severe perf ramifications in + // multi-tenant machines // The options are based on // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use // %20the%20Concurrent%20Low%20Pause%20Collector|outline diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index ed56d4bd44fe8..2915e664beffe 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -65,7 +65,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { override def isYarnMode(): Boolean = { true } // Return an appropriate (subclass) of Configuration. Creating a config initializes some Hadoop - // subsystems. Always create a new config, dont reuse yarnConf. + // subsystems. Always create a new config, don't reuse yarnConf. override def newConfiguration(conf: SparkConf): Configuration = new YarnConfiguration(super.newConfiguration(conf)) @@ -217,7 +217,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { // the hive configuration class is a subclass of Hadoop Configuration, so can be cast down // to a Configuration and used without reflection val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf") - // using the (Configuration, Class) constructor allows the current configuratin to be included + // using the (Configuration, Class) constructor allows the current configuration to be included // in the hive config. val ctor = hiveConfClass.getDeclaredConstructor(classOf[Configuration], classOf[Object].getClass) @@ -502,7 +502,7 @@ object YarnSparkHadoopUtil { /** * Getting the initial target number of executors depends on whether dynamic allocation is * enabled. - * If not using dynamic allocation it gets the number of executors reqeusted by the user. + * If not using dynamic allocation it gets the number of executors requested by the user. */ def getInitialTargetExecutorNumber( conf: SparkConf, diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index 1538ff75be5b3..05c1e1613dd35 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -78,7 +78,7 @@ private object YarnExternalShuffleDriver extends Logging with Matchers { s""" |Invalid command line: ${args.mkString(" ")} | - |Usage: ExternalShuffleDriver [result file] [registed exec file] + |Usage: ExternalShuffleDriver [result file] [registered exec file] """.stripMargin) // scalastyle:on println System.exit(1) From 31d069d4c2956306355d14087ca74ce1e6705217 Mon Sep 17 00:00:00 2001 From: Wilson Wu Date: Mon, 14 Mar 2016 09:13:29 +0000 Subject: [PATCH 19/21] [SPARK-13746][TESTS] stop using deprecated SynchronizedSet trait SynchronizedSet in package mutable is deprecated Author: Wilson Wu Closes #11580 from wilson888888888/spark-synchronizedset. --- .../apache/spark/ContextCleanerSuite.scala | 41 +++++++++++-------- .../kinesis/KinesisStreamSuite.scala | 22 ++++++---- 2 files changed, 39 insertions(+), 24 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index d1e806b2eb80a..e60678b300093 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark import java.lang.ref.WeakReference -import scala.collection.mutable.{HashSet, SynchronizedSet} +import scala.collection.mutable.HashSet import scala.language.existentials import scala.util.Random @@ -442,25 +442,25 @@ class CleanerTester( checkpointIds: Seq[Long] = Seq.empty) extends Logging { - val toBeCleanedRDDIds = new HashSet[Int] with SynchronizedSet[Int] ++= rddIds - val toBeCleanedShuffleIds = new HashSet[Int] with SynchronizedSet[Int] ++= shuffleIds - val toBeCleanedBroadcstIds = new HashSet[Long] with SynchronizedSet[Long] ++= broadcastIds - val toBeCheckpointIds = new HashSet[Long] with SynchronizedSet[Long] ++= checkpointIds + val toBeCleanedRDDIds = new HashSet[Int] ++= rddIds + val toBeCleanedShuffleIds = new HashSet[Int] ++= shuffleIds + val toBeCleanedBroadcstIds = new HashSet[Long] ++= broadcastIds + val toBeCheckpointIds = new HashSet[Long] ++= checkpointIds val isDistributed = !sc.isLocal val cleanerListener = new CleanerListener { def rddCleaned(rddId: Int): Unit = { - toBeCleanedRDDIds -= rddId + toBeCleanedRDDIds.synchronized { toBeCleanedRDDIds -= rddId } logInfo("RDD " + rddId + " cleaned") } def shuffleCleaned(shuffleId: Int): Unit = { - toBeCleanedShuffleIds -= shuffleId + toBeCleanedShuffleIds.synchronized { toBeCleanedShuffleIds -= shuffleId } logInfo("Shuffle " + shuffleId + " cleaned") } def broadcastCleaned(broadcastId: Long): Unit = { - toBeCleanedBroadcstIds -= broadcastId + toBeCleanedBroadcstIds.synchronized { toBeCleanedBroadcstIds -= broadcastId } logInfo("Broadcast " + broadcastId + " cleaned") } @@ -469,7 +469,7 @@ class CleanerTester( } def checkpointCleaned(rddId: Long): Unit = { - toBeCheckpointIds -= rddId + toBeCheckpointIds.synchronized { toBeCheckpointIds -= rddId } logInfo("checkpoint " + rddId + " cleaned") } } @@ -578,18 +578,27 @@ class CleanerTester( } private def uncleanedResourcesToString = { + val s1 = toBeCleanedRDDIds.synchronized { + toBeCleanedRDDIds.toSeq.sorted.mkString("[", ", ", "]") + } + val s2 = toBeCleanedShuffleIds.synchronized { + toBeCleanedShuffleIds.toSeq.sorted.mkString("[", ", ", "]") + } + val s3 = toBeCleanedBroadcstIds.synchronized { + toBeCleanedBroadcstIds.toSeq.sorted.mkString("[", ", ", "]") + } s""" - |\tRDDs = ${toBeCleanedRDDIds.toSeq.sorted.mkString("[", ", ", "]")} - |\tShuffles = ${toBeCleanedShuffleIds.toSeq.sorted.mkString("[", ", ", "]")} - |\tBroadcasts = ${toBeCleanedBroadcstIds.toSeq.sorted.mkString("[", ", ", "]")} + |\tRDDs = $s1 + |\tShuffles = $s2 + |\tBroadcasts = $s3 """.stripMargin } private def isAllCleanedUp = - toBeCleanedRDDIds.isEmpty && - toBeCleanedShuffleIds.isEmpty && - toBeCleanedBroadcstIds.isEmpty && - toBeCheckpointIds.isEmpty + toBeCleanedRDDIds.synchronized { toBeCleanedRDDIds.isEmpty } && + toBeCleanedShuffleIds.synchronized { toBeCleanedShuffleIds.isEmpty } && + toBeCleanedBroadcstIds.synchronized { toBeCleanedBroadcstIds.isEmpty } && + toBeCheckpointIds.synchronized { toBeCheckpointIds.isEmpty } private def getRDDBlocks(rddId: Int): Seq[BlockId] = { blockManager.master.getMatchingBlockIds( _ match { diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala index ca5d13da46e99..4460b6bccaa81 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala @@ -180,17 +180,20 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun Seconds(10), StorageLevel.MEMORY_ONLY, awsCredentials.getAWSAccessKeyId, awsCredentials.getAWSSecretKey) - val collected = new mutable.HashSet[Int] with mutable.SynchronizedSet[Int] + val collected = new mutable.HashSet[Int] stream.map { bytes => new String(bytes).toInt }.foreachRDD { rdd => - collected ++= rdd.collect() - logInfo("Collected = " + collected.mkString(", ")) + collected.synchronized { + collected ++= rdd.collect() + logInfo("Collected = " + collected.mkString(", ")) + } } ssc.start() val testData = 1 to 10 eventually(timeout(120 seconds), interval(10 second)) { testUtils.pushData(testData, aggregateTestData) - assert(collected === testData.toSet, "\nData received does not match data sent") + assert(collected.synchronized { collected === testData.toSet }, + "\nData received does not match data sent") } ssc.stop(stopSparkContext = false) } @@ -205,10 +208,12 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun stream shouldBe a [ReceiverInputDStream[_]] - val collected = new mutable.HashSet[Int] with mutable.SynchronizedSet[Int] + val collected = new mutable.HashSet[Int] stream.foreachRDD { rdd => - collected ++= rdd.collect() - logInfo("Collected = " + collected.mkString(", ")) + collected.synchronized { + collected ++= rdd.collect() + logInfo("Collected = " + collected.mkString(", ")) + } } ssc.start() @@ -216,7 +221,8 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun eventually(timeout(120 seconds), interval(10 second)) { testUtils.pushData(testData, aggregateTestData) val modData = testData.map(_ + 5) - assert(collected === modData.toSet, "\nData received does not match data sent") + assert(collected.synchronized { collected === modData.toSet }, + "\nData received does not match data sent") } ssc.stop(stopSparkContext = false) } From 250832c733602bcca034dd1fea6e325c55ccd1cd Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 14 Mar 2016 09:03:13 -0700 Subject: [PATCH 20/21] [SPARK-13207][SQL] Make partitioning discovery ignore _SUCCESS files. If a _SUCCESS appears in the inner partitioning dir, partition discovery will treat that _SUCCESS file as a data file. Then, partition discovery will fail because it finds that the dir structure is not valid. We should ignore those `_SUCCESS` files. In future, it is better to ignore all files/dirs starting with `_` or `.`. This PR does not make this change. I am thinking about making this change simple, so we can consider of getting it in branch 1.6. To ignore all files/dirs starting with `_` or `, the main change is to let ParquetRelation have another way to get metadata files. Right now, it relies on FileStatusCache's cachedLeafStatuses, which returns file statuses of both metadata files (e.g. metadata files used by parquet) and data files, which requires more changes. https://issues.apache.org/jira/browse/SPARK-13207 Author: Yin Huai Closes #11088 from yhuai/SPARK-13207. --- .../apache/spark/sql/sources/interfaces.scala | 30 +++++++++++++------ .../ParquetPartitionDiscoverySuite.scala | 23 ++++++++++++++ 2 files changed, 44 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index be2d98c46d3b0..601f944fb6363 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -522,7 +522,7 @@ class HDFSFileCatalog( } }.filterNot { status => val name = status.getPath.getName - name.toLowerCase == "_temporary" || name.startsWith(".") + HadoopFsRelation.shouldFilterOut(name) } val (dirs, files) = statuses.partition(_.isDirectory) @@ -616,6 +616,16 @@ class HDFSFileCatalog( * Helper methods for gathering metadata from HDFS. */ private[sql] object HadoopFsRelation extends Logging { + + /** Checks if we should filter out this path name. */ + def shouldFilterOut(pathName: String): Boolean = { + // TODO: We should try to filter out all files/dirs starting with "." or "_". + // The only reason that we are not doing it now is that Parquet needs to find those + // metadata files from leaf files returned by this methods. We should refactor + // this logic to not mix metadata files with data files. + pathName == "_SUCCESS" || pathName == "_temporary" || pathName.startsWith(".") + } + // We don't filter files/directories whose name start with "_" except "_temporary" here, as // specific data sources may take advantages over them (e.g. Parquet _metadata and // _common_metadata files). "_temporary" directories are explicitly ignored since failed @@ -624,19 +634,21 @@ private[sql] object HadoopFsRelation extends Logging { def listLeafFiles(fs: FileSystem, status: FileStatus): Array[FileStatus] = { logInfo(s"Listing ${status.getPath}") val name = status.getPath.getName.toLowerCase - if (name == "_temporary" || name.startsWith(".")) { + if (shouldFilterOut(name)) { Array.empty } else { // Dummy jobconf to get to the pathFilter defined in configuration val jobConf = new JobConf(fs.getConf, this.getClass()) val pathFilter = FileInputFormat.getInputPathFilter(jobConf) - if (pathFilter != null) { - val (dirs, files) = fs.listStatus(status.getPath, pathFilter).partition(_.isDirectory) - files ++ dirs.flatMap(dir => listLeafFiles(fs, dir)) - } else { - val (dirs, files) = fs.listStatus(status.getPath).partition(_.isDirectory) - files ++ dirs.flatMap(dir => listLeafFiles(fs, dir)) - } + val statuses = + if (pathFilter != null) { + val (dirs, files) = fs.listStatus(status.getPath, pathFilter).partition(_.isDirectory) + files ++ dirs.flatMap(dir => listLeafFiles(fs, dir)) + } else { + val (dirs, files) = fs.listStatus(status.getPath).partition(_.isDirectory) + files ++ dirs.flatMap(dir => listLeafFiles(fs, dir)) + } + statuses.filterNot(status => shouldFilterOut(status.getPath.getName)) } } 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 b74b9d3f3bbca..026191528ede4 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 @@ -706,6 +706,29 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } } + test("_SUCCESS should not break partitioning discovery") { + Seq(1, 32).foreach { threshold => + // We have two paths to list files, one at driver side, another one that we use + // a Spark job. We need to test both ways. + withSQLConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD.key -> threshold.toString) { + withTempPath { dir => + val tablePath = new File(dir, "table") + val df = (1 to 3).map(i => (i, i, i, i)).toDF("a", "b", "c", "d") + + df.write + .format("parquet") + .partitionBy("b", "c", "d") + .save(tablePath.getCanonicalPath) + + Files.touch(new File(s"${tablePath.getCanonicalPath}/b=1", "_SUCCESS")) + Files.touch(new File(s"${tablePath.getCanonicalPath}/b=1/c=1", "_SUCCESS")) + Files.touch(new File(s"${tablePath.getCanonicalPath}/b=1/c=1/d=1", "_SUCCESS")) + checkAnswer(sqlContext.read.format("parquet").load(tablePath.getCanonicalPath), df) + } + } + } + } + test("listConflictingPartitionColumns") { def makeExpectedMessage(colNameLists: Seq[String], paths: Seq[String]): String = { val conflictingColNameLists = colNameLists.zipWithIndex.map { case (list, index) => From 9a1680c2c85da4096bad71743debb2ccacdfd79f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 14 Mar 2016 09:57:24 -0700 Subject: [PATCH 21/21] [SPARK-13139][SQL] Follow-ups to #11573 Addressing outstanding comments in #11573. Jenkins, new test case in `DDLCommandSuite` Author: Andrew Or Closes #11667 from andrewor14/ddl-parser-followups. --- .../apache/spark/sql/execution/SparkQl.scala | 57 ++++++++---- .../command/AlterTableCommandParser.scala | 93 ++++++++++--------- .../spark/sql/execution/command/ddl.scala | 2 +- .../execution/command/DDLCommandSuite.scala | 10 +- 4 files changed, 94 insertions(+), 68 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala index d12dab567b00a..8dde308f96cc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkQl.scala @@ -34,8 +34,21 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly } /** - * For each node, extract properties in the form of a list ['key1', 'key2', 'key3', 'value'] - * into a pair (key1.key2.key3, value). + * For each node, extract properties in the form of a list + * ['key_part1', 'key_part2', 'key_part3', 'value'] + * into a pair (key_part1.key_part2.key_part3, value). + * + * Example format: + * + * TOK_TABLEPROPERTY + * :- 'k1' + * +- 'v1' + * TOK_TABLEPROPERTY + * :- 'k2' + * +- 'v2' + * TOK_TABLEPROPERTY + * :- 'k3' + * +- 'v3' */ private def extractProps( props: Seq[ASTNode], @@ -101,6 +114,16 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly } val props = dbprops.toSeq.flatMap { case Token("TOK_DATABASEPROPERTIES", Token("TOK_DBPROPLIST", propList) :: Nil) => + // Example format: + // + // TOK_DATABASEPROPERTIES + // +- TOK_DBPROPLIST + // :- TOK_TABLEPROPERTY + // : :- 'k1' + // : +- 'v1' + // :- TOK_TABLEPROPERTY + // :- 'k2' + // +- 'v2' extractProps(propList, "TOK_TABLEPROPERTY") case _ => parseFailed("Invalid CREATE DATABASE command", node) }.toMap @@ -112,16 +135,16 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly // Example format: // // TOK_CREATEFUNCTION - // :- db_name - // :- func_name - // :- alias - // +- TOK_RESOURCE_LIST - // :- TOK_RESOURCE_URI - // : :- TOK_JAR - // : +- '/path/to/jar' - // +- TOK_RESOURCE_URI - // :- TOK_FILE - // +- 'path/to/file' + // :- db_name + // :- func_name + // :- alias + // +- TOK_RESOURCE_LIST + // :- TOK_RESOURCE_URI + // : :- TOK_JAR + // : +- '/path/to/jar' + // +- TOK_RESOURCE_URI + // :- TOK_FILE + // +- 'path/to/file' val (funcNameArgs, otherArgs) = args.partition { case Token("TOK_RESOURCE_LIST", _) => false case Token("TOK_TEMPORARY", _) => false @@ -139,9 +162,9 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly } // Extract other keywords, if they exist val Seq(rList, temp) = getClauses(Seq("TOK_RESOURCE_LIST", "TOK_TEMPORARY"), otherArgs) - val resourcesMap = rList.toSeq.flatMap { - case Token("TOK_RESOURCE_LIST", resources) => - resources.map { + val resources: Seq[(String, String)] = rList.toSeq.flatMap { + case Token("TOK_RESOURCE_LIST", resList) => + resList.map { case Token("TOK_RESOURCE_URI", rType :: Token(rPath, Nil) :: Nil) => val resourceType = rType match { case Token("TOK_JAR", Nil) => "jar" @@ -153,8 +176,8 @@ private[sql] class SparkQl(conf: ParserConf = SimpleParserConf()) extends Cataly case _ => parseFailed("Invalid CREATE FUNCTION command", node) } case _ => parseFailed("Invalid CREATE FUNCTION command", node) - }.toMap - CreateFunction(funcName, alias, resourcesMap, temp.isDefined)(node.source) + } + CreateFunction(funcName, alias, resources, temp.isDefined)(node.source) case Token("TOK_ALTERTABLE", alterTableArgs) => AlterTableCommandParser.parse(node) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala index 58639275c111b..9fbe6db467ffa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommandParser.scala @@ -55,20 +55,22 @@ object AlterTableCommandParser { /** * Extract partition spec from the given [[ASTNode]] as a map, assuming it exists. * - * Expected format: - * +- TOK_PARTSPEC - * :- TOK_PARTVAL - * : :- dt - * : +- '2008-08-08' - * +- TOK_PARTVAL - * :- country - * +- 'us' + * Example format: + * + * TOK_PARTSPEC + * :- TOK_PARTVAL + * : :- dt + * : +- '2008-08-08' + * +- TOK_PARTVAL + * :- country + * +- 'us' */ private def parsePartitionSpec(node: ASTNode): Map[String, String] = { node match { case Token("TOK_PARTSPEC", partitions) => partitions.map { // Note: sometimes there's a "=", "<" or ">" between the key and the value + // (e.g. when dropping all partitions with value > than a certain constant) case Token("TOK_PARTVAL", ident :: conj :: constant :: Nil) => (cleanAndUnquoteString(ident.text), cleanAndUnquoteString(constant.text)) case Token("TOK_PARTVAL", ident :: constant :: Nil) => @@ -86,15 +88,16 @@ object AlterTableCommandParser { /** * Extract table properties from the given [[ASTNode]] as a map, assuming it exists. * - * Expected format: - * +- TOK_TABLEPROPERTIES - * +- TOK_TABLEPROPLIST - * :- TOK_TABLEPROPERTY - * : :- 'test' - * : +- 'value' - * +- TOK_TABLEPROPERTY - * :- 'comment' - * +- 'new_comment' + * Example format: + * + * TOK_TABLEPROPERTIES + * +- TOK_TABLEPROPLIST + * :- TOK_TABLEPROPERTY + * : :- 'test' + * : +- 'value' + * +- TOK_TABLEPROPERTY + * :- 'comment' + * +- 'new_comment' */ private def extractTableProps(node: ASTNode): Map[String, String] = { node match { @@ -209,21 +212,21 @@ object AlterTableCommandParser { Token("TOK_TABCOLNAME", colNames) :: colValues :: rest) :: Nil) :: _ => // Example format: // - // +- TOK_ALTERTABLE_SKEWED - // :- TOK_TABLESKEWED - // : :- TOK_TABCOLNAME - // : : :- dt - // : : +- country - // :- TOK_TABCOLVALUE_PAIR - // : :- TOK_TABCOLVALUES - // : : :- TOK_TABCOLVALUE - // : : : :- '2008-08-08' - // : : : +- 'us' - // : :- TOK_TABCOLVALUES - // : : :- TOK_TABCOLVALUE - // : : : :- '2009-09-09' - // : : : +- 'uk' - // +- TOK_STOREASDIR + // TOK_ALTERTABLE_SKEWED + // :- TOK_TABLESKEWED + // : :- TOK_TABCOLNAME + // : : :- dt + // : : +- country + // :- TOK_TABCOLVALUE_PAIR + // : :- TOK_TABCOLVALUES + // : : :- TOK_TABCOLVALUE + // : : : :- '2008-08-08' + // : : : +- 'us' + // : :- TOK_TABCOLVALUES + // : : :- TOK_TABCOLVALUE + // : : : :- '2009-09-09' + // : : : +- 'uk' + // +- TOK_STOREASDIR val names = colNames.map { n => cleanAndUnquoteString(n.text) } val values = colValues match { case Token("TOK_TABCOLVALUE", vals) => @@ -260,20 +263,20 @@ object AlterTableCommandParser { case Token("TOK_ALTERTABLE_SKEWED_LOCATION", Token("TOK_SKEWED_LOCATIONS", Token("TOK_SKEWED_LOCATION_LIST", locationMaps) :: Nil) :: Nil) :: _ => - // Expected format: + // Example format: // - // +- TOK_ALTERTABLE_SKEWED_LOCATION - // +- TOK_SKEWED_LOCATIONS - // +- TOK_SKEWED_LOCATION_LIST - // :- TOK_SKEWED_LOCATION_MAP - // : :- 'col1' - // : +- 'loc1' - // +- TOK_SKEWED_LOCATION_MAP - // :- TOK_TABCOLVALUES - // : +- TOK_TABCOLVALUE - // : :- 'col2' - // : +- 'col3' - // +- 'loc2' + // TOK_ALTERTABLE_SKEWED_LOCATION + // +- TOK_SKEWED_LOCATIONS + // +- TOK_SKEWED_LOCATION_LIST + // :- TOK_SKEWED_LOCATION_MAP + // : :- 'col1' + // : +- 'loc1' + // +- TOK_SKEWED_LOCATION_MAP + // :- TOK_TABCOLVALUES + // : +- TOK_TABCOLVALUE + // : :- 'col2' + // : +- 'col3' + // +- 'loc2' val skewedMaps = locationMaps.flatMap { case Token("TOK_SKEWED_LOCATION_MAP", col :: loc :: Nil) => col match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 9df58d214a504..3fb2e34101a68 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -55,7 +55,7 @@ case class CreateDatabase( case class CreateFunction( functionName: String, alias: String, - resourcesMap: Map[String, String], + resources: Seq[(String, String)], isTemp: Boolean)(sql: String) extends NativeDDLCommand(sql) with Logging diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala index 0d632a8a130ed..6f1eea273fafa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLCommandSuite.scala @@ -48,26 +48,26 @@ class DDLCommandSuite extends PlanTest { val sql1 = """ |CREATE TEMPORARY FUNCTION helloworld as - |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar', - |FILE 'path/to/file' + |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1', + |JAR '/path/to/jar2' """.stripMargin val sql2 = """ |CREATE FUNCTION hello.world as |'com.matthewrathbone.example.SimpleUDFExample' USING ARCHIVE '/path/to/archive', - |FILE 'path/to/file' + |FILE '/path/to/file' """.stripMargin val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val expected1 = CreateFunction( "helloworld", "com.matthewrathbone.example.SimpleUDFExample", - Map("jar" -> "/path/to/jar", "file" -> "path/to/file"), + Seq(("jar", "/path/to/jar1"), ("jar", "/path/to/jar2")), isTemp = true)(sql1) val expected2 = CreateFunction( "hello.world", "com.matthewrathbone.example.SimpleUDFExample", - Map("archive" -> "/path/to/archive", "file" -> "path/to/file"), + Seq(("archive", "/path/to/archive"), ("file", "/path/to/file")), isTemp = false)(sql2) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2)