Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
21 changes: 17 additions & 4 deletions docs/streaming-programming-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -878,6 +878,12 @@ This is applied on a DStream containing words (say, the `pairs` DStream containi
val runningCounts = pairs.updateStateByKey[Int](updateFunction _)
{% endhighlight %}

The update function will be called for each word, with `newValues` having a sequence of 1's (from
the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete
Scala code, take a look at the example
[StatefulNetworkWordCount.scala]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache
/spark/examples/streaming/StatefulNetworkWordCount.scala).

</div>
<div data-lang="java" markdown="1">

Expand All @@ -899,6 +905,13 @@ This is applied on a DStream containing words (say, the `pairs` DStream containi
JavaPairDStream<String, Integer> runningCounts = pairs.updateStateByKey(updateFunction);
{% endhighlight %}

The update function will be called for each word, with `newValues` having a sequence of 1's (from
the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete
Java code, take a look at the example
[JavaStatefulNetworkWordCount.java]({{site
.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming
/JavaStatefulNetworkWordCount.java).

</div>
<div data-lang="python" markdown="1">

Expand All @@ -916,14 +929,14 @@ This is applied on a DStream containing words (say, the `pairs` DStream containi
runningCounts = pairs.updateStateByKey(updateFunction)
{% endhighlight %}

</div>
</div>

The update function will be called for each word, with `newValues` having a sequence of 1's (from
the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete
Scala code, take a look at the example
Python code, take a look at the example
[stateful_network_wordcount.py]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/stateful_network_wordcount.py).

</div>
</div>

Note that using `updateStateByKey` requires the checkpoint directory to be configured, which is
discussed in detail in the [checkpointing](#checkpointing) section.

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,115 @@
/*
* 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.examples.streaming;

import java.util.Arrays;
import java.util.List;
import java.util.regex.Pattern;

import scala.Tuple2;

import com.google.common.base.Optional;
import com.google.common.collect.Lists;

import org.apache.spark.HashPartitioner;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.StorageLevels;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.Function2;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.streaming.Durations;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
import org.apache.spark.streaming.api.java.JavaReceiverInputDStream;
import org.apache.spark.streaming.api.java.JavaStreamingContext;

/**
* Counts words cumulatively in UTF8 encoded, '\n' delimited text received from the network every
* second starting with initial value of word count.
* Usage: JavaStatefulNetworkWordCount <hostname> <port>
* <hostname> and <port> describe the TCP server that Spark Streaming would connect to receive
* data.
* <p/>
* To run this on your local machine, you need to first run a Netcat server
* `$ nc -lk 9999`
* and then run the example
* `$ bin/run-example
* org.apache.spark.examples.streaming.JavaStatefulNetworkWordCount localhost 9999`
*/
public class JavaStatefulNetworkWordCount {
private static final Pattern SPACE = Pattern.compile(" ");

public static void main(String[] args) {
if (args.length < 2) {
System.err.println("Usage: JavaStatefulNetworkWordCount <hostname> <port>");
System.exit(1);
}

StreamingExamples.setStreamingLogLevels();

// Update the cumulative count function
final Function2<List<Integer>, Optional<Integer>, Optional<Integer>> updateFunction =
new Function2<List<Integer>, Optional<Integer>, Optional<Integer>>() {
@Override
public Optional<Integer> call(List<Integer> values, Optional<Integer> state) {
Integer newSum = state.or(0);
for (Integer value : values) {
newSum += value;
}
return Optional.of(newSum);
}
};

// Create the context with a 1 second batch size
SparkConf sparkConf = new SparkConf().setAppName("JavaStatefulNetworkWordCount");
JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, Durations.seconds(1));
ssc.checkpoint(".");

// Initial RDD input to updateStateByKey
List<Tuple2<String, Integer>> tuples = Arrays.asList(new Tuple2<String, Integer>("hello", 1),
new Tuple2<String, Integer>("world", 1));
JavaPairRDD<String, Integer> initialRDD = ssc.sc().parallelizePairs(tuples);

JavaReceiverInputDStream<String> lines = ssc.socketTextStream(
args[0], Integer.parseInt(args[1]), StorageLevels.MEMORY_AND_DISK_SER_2);

JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
@Override
public Iterable<String> call(String x) {
return Lists.newArrayList(SPACE.split(x));
}
});

JavaPairDStream<String, Integer> wordsDstream = words.mapToPair(
new PairFunction<String, String, Integer>() {
@Override
public Tuple2<String, Integer> call(String s) {
return new Tuple2<String, Integer>(s, 1);
}
});

// This will give a Dstream made of state (which is the cumulative count of the words)
JavaPairDStream<String, Integer> stateDstream = wordsDstream.updateStateByKey(updateFunction,
new HashPartitioner(ssc.sc().defaultParallelism()), initialRDD);

stateDstream.print();
ssc.start();
ssc.awaitTermination();
}
}
6 changes: 6 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -1083,6 +1083,12 @@
<artifactId>scala-maven-plugin</artifactId>
<version>3.2.0</version>
<executions>
<execution>
<id>eclipse-add-source</id>
<goals>
<goal>add-source</goal>
</goals>
</execution>
<execution>
<id>scala-compile-first</id>
<phase>process-resources</phase>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,7 @@ private[sql] class DefaultSource
case SaveMode.Append =>
sys.error(s"Append mode is not supported by ${this.getClass.getCanonicalName}")
case SaveMode.Overwrite =>
fs.delete(filesystemPath, true)
//fs.delete(filesystemPath, true)
true
case SaveMode.ErrorIfExists =>
sys.error(s"path $path already exists.")
Expand All @@ -76,12 +76,18 @@ private[sql] class DefaultSource
} else {
true
}
if (doSave) {
val relation = if (doSave) {
// Only save data when the save mode is not ignore.
data.toJSON.saveAsTextFile(path)
//data.toJSON.saveAsTextFile(path)
val createdRelation = createRelation(sqlContext,parameters, data.schema)
createdRelation.asInstanceOf[JSONRelation].insert(data, true)

createdRelation
} else {
createRelation(sqlContext, parameters, data.schema)
}

createRelation(sqlContext, parameters, data.schema)
relation
}
}

Expand All @@ -92,7 +98,15 @@ private[sql] case class JSONRelation(
@transient val sqlContext: SQLContext)
extends TableScan with InsertableRelation {
// TODO: Support partitioned JSON relation.
private def baseRDD = sqlContext.sparkContext.textFile(path)
val filesystemPath = new Path(path)
val fs = filesystemPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration)
// TableScan can support base on ordinary file, but InsertableRelation only base on directory.
val newPath = if (fs.exists(filesystemPath) && fs.getFileStatus(filesystemPath).isFile()) {
filesystemPath
} else {
new Path(filesystemPath.toUri.toString,"*")
}
private def baseRDD = sqlContext.sparkContext.textFile(newPath.toUri.toString)

override val schema = userSpecifiedSchema.getOrElse(
JsonRDD.nullTypeToStringType(
Expand All @@ -104,21 +118,35 @@ private[sql] case class JSONRelation(
override def buildScan() =
JsonRDD.jsonStringToRow(baseRDD, schema, sqlContext.conf.columnNameOfCorruptRecord)

private def isTemporaryFile(file: Path): Boolean = {
file.getName == "_temporary"
}

override def insert(data: DataFrame, overwrite: Boolean) = {
val filesystemPath = new Path(path)
val fs = filesystemPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration)

// If the path exists, it must be a directory, error for not.
// Otherwise we create a directory with the path name.
if (fs.exists(filesystemPath) && !fs.getFileStatus(filesystemPath).isDirectory) {
sys.error("a CREATE [TEMPORARY] TABLE AS SELECT statement need the path must be directory")
}

if (overwrite) {
val temporaryPath = new Path(path, "_temporary")
val dataPath = new Path(path, "data")
// Write the data.
data.toJSON.saveAsTextFile(temporaryPath.toUri.toString)
val pathsToDelete = fs.listStatus(filesystemPath).filter(
f => !isTemporaryFile(f.getPath)).map(_.getPath)

try {
fs.delete(filesystemPath, true)
pathsToDelete.foreach(fs.delete(_,true))
} catch {
case e: IOException =>
throw new IOException(
s"Unable to clear output directory ${filesystemPath.toString} prior"
+ s" to INSERT OVERWRITE a JSON table:\n${e.toString}")
s"Unable to delete original data in directory ${filesystemPath.toString} when"
+ s" run INSERT OVERWRITE a JSON table:\n${e.toString}")
}
// Write the data.
data.toJSON.saveAsTextFile(path)
fs.rename(temporaryPath,dataPath)
// Right now, we assume that the schema is not changed. We will not update the schema.
// schema = data.schema
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,10 +29,13 @@ class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll {
import caseInsensisitiveContext._

var path: File = null
var existPath: File = null

override def beforeAll(): Unit = {
path = util.getTempFilePath("jsonCTAS").getCanonicalFile
existPath = util.getTempFilePath("existJsonCTAS").getCanonicalFile
val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}"""))
rdd.saveAsTextFile(existPath.toURI.toString)
jsonRDD(rdd).registerTempTable("jt")
}

Expand Down Expand Up @@ -62,6 +65,34 @@ class CreateTableAsSelectSuite extends DataSourceTest with BeforeAndAfterAll {
dropTempTable("jsonTable")
}

test("INSERT OVERWRITE with the source and destination point to the same table") {
sql(
s"""
|CREATE TEMPORARY TABLE jsonTable1
|USING org.apache.spark.sql.json.DefaultSource
|OPTIONS (
| path '${existPath.toString}'
|)
""".stripMargin)

sql(
s"""
|CREATE TEMPORARY TABLE jsonTable2
|USING org.apache.spark.sql.json.DefaultSource
|OPTIONS (
| path '${existPath.toString}'
|) AS
|SELECT a, b FROM jsonTable1
""".stripMargin)

checkAnswer(
sql("SELECT a, b FROM jsonTable2"),
sql("SELECT a, b FROM jt").collect())

dropTempTable("jsonTable1")
dropTempTable("jsonTable2")
}

test("create a table, drop it and create another one with the same name") {
sql(
s"""
Expand Down