LuciferYang commented on code in PR #40783: URL: https://github.com/apache/spark/pull/40783#discussion_r1168322731
########## connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala: ########## @@ -0,0 +1,275 @@ +/* + * 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.streaming + +import scala.collection.JavaConverters._ + +import org.apache.spark.annotation.Evolving +import org.apache.spark.connect.proto.Read.DataSource +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.types.StructType + +/** + * Interface used to load a streaming `Dataset` from external storage systems (e.g. file systems, + * key-value stores, etc). Use `SparkSession.readStream` to access this. + * + * @since 3.5.0 + */ +@Evolving +final class DataStreamReader private[sql] (sparkSession: SparkSession) { + + /** + * Specifies the input data source format. + * + * @since 3.5.0 + */ + def format(source: String): DataStreamReader = { + sourceBuilder.setFormat(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 loading. + * + * @since 3.5.0 + */ + def schema(schema: StructType): DataStreamReader = { + if (schema != null) { + sourceBuilder.setSchema(schema.json) // Use json. DDL does not retail all the attributes. + } + this + } + + /** + * Specifies the schema by using the input DDL-formatted string. 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 loading. + * + * @since 3.5.0 + */ + def schema(schemaString: String): DataStreamReader = { + sourceBuilder.setSchema(schemaString) + this + } + + /** + * Adds an input option for the underlying data source. + * + * @since 3.5.0 + */ + def option(key: String, value: String): DataStreamReader = { + sourceBuilder.putOptions(key, value) + this + } + + /** + * Adds an input option for the underlying data source. + * + * @since 3.5.0 + */ + def option(key: String, value: Boolean): DataStreamReader = option(key, value.toString) + + /** + * Adds an input option for the underlying data source. + * + * @since 3.5.0 + */ + def option(key: String, value: Long): DataStreamReader = option(key, value.toString) + + /** + * Adds an input option for the underlying data source. + * + * @since 3.5.0 + */ + def option(key: String, value: Double): DataStreamReader = option(key, value.toString) + + /** + * (Scala-specific) Adds input options for the underlying data source. + * + * @since 3.5.0 + */ + def options(options: scala.collection.Map[String, String]): DataStreamReader = { + this.options(options.asJava) + this + } + + /** + * (Java-specific) Adds input options for the underlying data source. + * + * @since 3.5.0 + */ + def options(options: java.util.Map[String, String]): DataStreamReader = { + sourceBuilder.putAllOptions(options) + this + } + + /** + * Loads input data stream in as a `DataFrame`, for data streams that don't require a path (e.g. + * external key-value stores). + * + * @since 3.5.0 + */ + def load(): DataFrame = { + sparkSession.newDataFrame { relationBuilder => + relationBuilder.getReadBuilder + .setIsStreaming(true) + .setDataSource(sourceBuilder.build()) + } + } + + /** + * Loads input in as a `DataFrame`, for data streams that read from some path. + * + * @since 3.5.0 + */ + def load(path: String): DataFrame = { + sourceBuilder.clearPaths() Review Comment: Hmm.. isthis considering the following usage way ```scala val rs = spark.readStream ..... val parquetDf = rs.parquet("parquet-path") val orcDf = rs.orc("orc-path") ``` ? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
