forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
Showing
15 changed files
with
666 additions
and
7 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
137 changes: 137 additions & 0 deletions
137
sql/core/src/main/scala/org/apache/spark/sql/DataStreamReader.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,137 @@ | ||
/* | ||
* 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 | ||
|
||
import org.apache.spark.sql.execution.streaming.StreamingRelation | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import org.apache.hadoop.util.StringUtils | ||
|
||
import org.apache.spark.Logging | ||
import org.apache.spark.annotation.Experimental | ||
import org.apache.spark.sql.execution.datasources.{LogicalRelation, ResolvedDataSource} | ||
import org.apache.spark.sql.types.StructType | ||
|
||
/** | ||
* :: Experimental :: | ||
* An interface to reading streaming data. Use `sqlContext.stream` to access these methods. | ||
*/ | ||
@Experimental | ||
class DataStreamReader private[sql](sqlContext: SQLContext) extends Logging { | ||
|
||
/** | ||
* Specifies the input data source format. | ||
* | ||
* @since 1.4.0 | ||
*/ | ||
def format(source: String): DataStreamReader = { | ||
this.source = source | ||
this | ||
} | ||
|
||
/** | ||
* Specifies the input schema. Some data sources (e.g. JSON) can infer the input schema | ||
* automatically from data. By specifying the schema here, the underlying data source can | ||
* skip the schema inference step, and thus speed up data reading. | ||
* | ||
* @since 1.4.0 | ||
*/ | ||
def schema(schema: StructType): DataStreamReader = { | ||
this.userSpecifiedSchema = Option(schema) | ||
this | ||
} | ||
|
||
/** | ||
* Adds an input option for the underlying data source. | ||
* | ||
* @since 1.4.0 | ||
*/ | ||
def option(key: String, value: String): DataStreamReader = { | ||
this.extraOptions += (key -> value) | ||
this | ||
} | ||
|
||
/** | ||
* (Scala-specific) Adds input options for the underlying data source. | ||
* | ||
* @since 1.4.0 | ||
*/ | ||
def options(options: scala.collection.Map[String, String]): DataStreamReader = { | ||
this.extraOptions ++= options | ||
this | ||
} | ||
|
||
/** | ||
* Adds input options for the underlying data source. | ||
* | ||
* @since 1.4.0 | ||
*/ | ||
def options(options: java.util.Map[String, String]): DataStreamReader = { | ||
this.options(options.asScala) | ||
this | ||
} | ||
|
||
/** | ||
* Loads input in as a [[DataFrame]], for data sources that require a path (e.g. data backed by | ||
* a local or distributed file system). | ||
* | ||
* @since 1.4.0 | ||
*/ | ||
// TODO: Remove this one in Spark 2.0. | ||
def open(path: String): DataFrame = { | ||
option("path", path).open() | ||
} | ||
|
||
/** | ||
* Loads input in as a [[DataFrame]], for data sources that don't require a path (e.g. external | ||
* key-value stores). | ||
* | ||
* @since 1.4.0 | ||
*/ | ||
def open(): DataFrame = { | ||
val resolved = ResolvedDataSource.createSource( | ||
sqlContext, | ||
userSpecifiedSchema = userSpecifiedSchema, | ||
providerName = source, | ||
options = extraOptions.toMap) | ||
DataFrame(sqlContext, StreamingRelation(resolved)) | ||
} | ||
|
||
/** | ||
* Loads input in as a [[DataFrame]], for data sources that support multiple paths. | ||
* Only works if the source is a HadoopFsRelationProvider. | ||
* | ||
* @since 1.6.0 | ||
*/ | ||
@scala.annotation.varargs | ||
def open(paths: String*): DataFrame = { | ||
option("paths", paths.map(StringUtils.escapeString(_, '\\', ',')).mkString(",")).open() | ||
} | ||
|
||
/////////////////////////////////////////////////////////////////////////////////////// | ||
// Builder pattern config options | ||
/////////////////////////////////////////////////////////////////////////////////////// | ||
|
||
private var source: String = sqlContext.conf.defaultDataSourceName | ||
|
||
private var userSpecifiedSchema: Option[StructType] = None | ||
|
||
private var extraOptions = new scala.collection.mutable.HashMap[String, String] | ||
|
||
} |
155 changes: 155 additions & 0 deletions
155
sql/core/src/main/scala/org/apache/spark/sql/DataStreamWriter.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,155 @@ | ||
/* | ||
* 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 | ||
|
||
import java.util.Properties | ||
|
||
import org.apache.spark.sql.execution.streaming.{Offset, Sink, Batch, StreamExecution} | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import org.apache.spark.annotation.Experimental | ||
import org.apache.spark.sql.catalyst.{SqlParser, TableIdentifier} | ||
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedRelation} | ||
import org.apache.spark.sql.catalyst.plans.logical.{Project, InsertIntoTable} | ||
import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils | ||
import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, ResolvedDataSource} | ||
import org.apache.spark.sql.sources.HadoopFsRelation | ||
|
||
|
||
/** | ||
* :: Experimental :: | ||
* Interface used to start a streaming query query execution. | ||
* | ||
* @since 2.0.0 | ||
*/ | ||
@Experimental | ||
final class DataStreamWriter private[sql](df: DataFrame) { | ||
|
||
/** | ||
* Specifies the underlying output data source. Built-in options include "parquet", "json", etc. | ||
* | ||
* @since 1.4.0 | ||
*/ | ||
def format(source: String): DataStreamWriter = { | ||
this.source = source | ||
this | ||
} | ||
|
||
/** | ||
* Adds an output option for the underlying data source. | ||
* | ||
* @since 1.4.0 | ||
*/ | ||
def option(key: String, value: String): DataStreamWriter = { | ||
this.extraOptions += (key -> value) | ||
this | ||
} | ||
|
||
/** | ||
* (Scala-specific) Adds output options for the underlying data source. | ||
* | ||
* @since 1.4.0 | ||
*/ | ||
def options(options: scala.collection.Map[String, String]): DataStreamWriter = { | ||
this.extraOptions ++= options | ||
this | ||
} | ||
|
||
/** | ||
* Adds output options for the underlying data source. | ||
* | ||
* @since 1.4.0 | ||
*/ | ||
def options(options: java.util.Map[String, String]): DataStreamWriter = { | ||
this.options(options.asScala) | ||
this | ||
} | ||
|
||
/** | ||
* Partitions the output by the given columns on the file system. If specified, the output is | ||
* laid out on the file system similar to Hive's partitioning scheme. | ||
* | ||
* This was initially applicable for Parquet but in 1.5+ covers JSON, text, ORC and avro as well. | ||
* | ||
* @since 1.4.0 | ||
*/ | ||
@scala.annotation.varargs | ||
def partitionBy(colNames: String*): DataStreamWriter = { | ||
this.partitioningColumns = Option(colNames) | ||
this | ||
} | ||
|
||
/** | ||
* Saves the content of the [[DataFrame]] at the specified path. | ||
* | ||
* @since 1.4.0 | ||
*/ | ||
def start(path: String): StandingQuery = { | ||
this.extraOptions += ("path" -> path) | ||
start() | ||
} | ||
|
||
/** | ||
* Saves the content of the [[DataFrame]] as the specified table. | ||
* | ||
* @since 1.4.0 | ||
*/ | ||
def start(): StandingQuery = { | ||
val sink = ResolvedDataSource.createSink( | ||
df.sqlContext, | ||
source, | ||
extraOptions.toMap) | ||
|
||
new StreamExecution(df.sqlContext, df.logicalPlan, sink) | ||
} | ||
|
||
def foreach(f: Batch => Unit): StandingQuery = { | ||
val checkpoint = extraOptions.get("checkpoint") | ||
val sink = new Sink { | ||
override def currentOffset: Option[Offset] = None | ||
override def addBatch(batch: Batch): Unit = f(batch) | ||
} | ||
|
||
new StreamExecution(df.sqlContext, df.logicalPlan, sink) | ||
} | ||
|
||
private def normalizedParCols: Option[Seq[String]] = partitioningColumns.map { parCols => | ||
parCols.map { col => | ||
df.logicalPlan.output | ||
.map(_.name) | ||
.find(df.sqlContext.analyzer.resolver(_, col)) | ||
.getOrElse(throw new AnalysisException(s"Partition column $col not found in existing " + | ||
s"columns (${df.logicalPlan.output.map(_.name).mkString(", ")})")) | ||
} | ||
} | ||
|
||
|
||
/////////////////////////////////////////////////////////////////////////////////////// | ||
// Builder pattern config options | ||
/////////////////////////////////////////////////////////////////////////////////////// | ||
|
||
private var source: String = df.sqlContext.conf.defaultDataSourceName | ||
|
||
private var mode: SaveMode = SaveMode.ErrorIfExists | ||
|
||
private var extraOptions = new scala.collection.mutable.HashMap[String, String] | ||
|
||
private var partitioningColumns: Option[Seq[String]] = None | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
22 changes: 22 additions & 0 deletions
22
sql/core/src/main/scala/org/apache/spark/sql/StandingQuery.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,22 @@ | ||
/* | ||
* 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 | ||
|
||
trait StandingQuery { | ||
def stop(): Unit | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.