[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-30 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237995405
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,51 @@
+/*
+ * 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.sources.v2;
--- End diff --

I just went to make this change, but it requires moving any SQL class from 
catalyst referenced by the API into the API module as well... Let's discuss the 
options more on the dev list thread.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-30 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237966188
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java ---
@@ -0,0 +1,68 @@
+/*
+ * 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.sources.v2.reader;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.sources.v2.SupportsBatchRead;
+import org.apache.spark.sql.sources.v2.Table;
+
+/**
+ * A logical representation of a data source scan. This interface is used 
to provide logical
+ * information, like what the actual read schema is.
+ * 
+ * This logical representation is shared between batch scan, micro-batch 
streaming scan and
+ * continuous streaming scan. Data sources must implement the 
corresponding methods in this
+ * interface, to match what the table promises to support. For example, 
{@link #toBatch()} must be
+ * implemented, if the {@link Table} that creates this {@link Scan} 
implements
+ * {@link SupportsBatchRead}.
+ * 
+ */
+@Evolving
+public interface Scan {
+
+  /**
+   * Returns the actual schema of this data source scan, which may be 
different from the physical
+   * schema of the underlying storage, as column pruning or other 
optimizations may happen.
+   */
+  StructType readSchema();
+
+  /**
+   * A description string of this scan, which may includes information 
like: what filters are
+   * configured for this scan, what's the value of some important options 
like path, etc. The
+   * description doesn't need to include {@link #readSchema()}, as Spark 
already knows it.
+   * 
+   * By default this returns the class name of the implementation. Please 
override it to provide a
+   * meaningful description.
+   * 
+   */
+  default String description() {
--- End diff --

What about adding `pushedFilters` that defaults to `new Filter[0]`? Then 
users should override that to add filters to the description, if they are 
pushed. I think a Scan should be able to report its options, especially those 
that distinguish it from other scans, like pushed filters.

I guess we could have some wrapper around the user-provided Scan that holds 
the Scan options. I would want to standardize that instead of doing it in every 
scan exec node.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-30 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/23086


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-29 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237734357
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java ---
@@ -0,0 +1,68 @@
+/*
+ * 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.sources.v2.reader;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.sources.v2.SupportsBatchRead;
+import org.apache.spark.sql.sources.v2.Table;
+
+/**
+ * A logical representation of a data source scan. This interface is used 
to provide logical
+ * information, like what the actual read schema is.
+ * 
+ * This logical representation is shared between batch scan, micro-batch 
streaming scan and
+ * continuous streaming scan. Data sources must implement the 
corresponding methods in this
+ * interface, to match what the table promises to support. For example, 
{@link #toBatch()} must be
+ * implemented, if the {@link Table} that creates this {@link Scan} 
implements
+ * {@link SupportsBatchRead}.
+ * 
+ */
+@Evolving
+public interface Scan {
+
+  /**
+   * Returns the actual schema of this data source scan, which may be 
different from the physical
+   * schema of the underlying storage, as column pruning or other 
optimizations may happen.
+   */
+  StructType readSchema();
+
+  /**
+   * A description string of this scan, which may includes information 
like: what filters are
+   * configured for this scan, what's the value of some important options 
like path, etc. The
+   * description doesn't need to include {@link #readSchema()}, as Spark 
already knows it.
+   * 
+   * By default this returns the class name of the implementation. Please 
override it to provide a
+   * meaningful description.
+   * 
+   */
+  default String description() {
--- End diff --

Since this is an interface, and filter pushdown is optional, I'm not sure 
how to report `pushedFilters` here.

The read schema is always reported, see 
`DataSourceV2ScanExec.simpleString`. Maybe we should still keep `pushedFilters` 
in `DataSourceV2ScanExec`, and display it in the plan string format. What do 
you think? 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-29 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237670228
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
 ---
@@ -22,86 +22,56 @@ import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical
 import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, 
WholeStageCodegenExec}
-import org.apache.spark.sql.execution.streaming.continuous._
-import org.apache.spark.sql.sources.v2.DataSourceV2
 import org.apache.spark.sql.sources.v2.reader._
-import 
org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory,
 ContinuousReadSupport, MicroBatchReadSupport}
 
 /**
- * Physical plan node for scanning data from a data source.
+ * Physical plan node for scanning a batch of data from a data source.
  */
 case class DataSourceV2ScanExec(
 output: Seq[AttributeReference],
-@transient source: DataSourceV2,
-@transient options: Map[String, String],
-@transient pushedFilters: Seq[Expression],
-@transient readSupport: ReadSupport,
-@transient scanConfig: ScanConfig)
-  extends LeafExecNode with DataSourceV2StringFormat with 
ColumnarBatchScan {
+scanDesc: String,
+@transient batch: Batch)
--- End diff --

Sounds good to me.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-29 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237670099
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java ---
@@ -0,0 +1,68 @@
+/*
+ * 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.sources.v2.reader;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.sources.v2.SupportsBatchRead;
+import org.apache.spark.sql.sources.v2.Table;
+
+/**
+ * A logical representation of a data source scan. This interface is used 
to provide logical
+ * information, like what the actual read schema is.
+ * 
+ * This logical representation is shared between batch scan, micro-batch 
streaming scan and
+ * continuous streaming scan. Data sources must implement the 
corresponding methods in this
+ * interface, to match what the table promises to support. For example, 
{@link #toBatch()} must be
+ * implemented, if the {@link Table} that creates this {@link Scan} 
implements
+ * {@link SupportsBatchRead}.
+ * 
+ */
+@Evolving
+public interface Scan {
+
+  /**
+   * Returns the actual schema of this data source scan, which may be 
different from the physical
+   * schema of the underlying storage, as column pruning or other 
optimizations may happen.
+   */
+  StructType readSchema();
+
+  /**
+   * A description string of this scan, which may includes information 
like: what filters are
+   * configured for this scan, what's the value of some important options 
like path, etc. The
+   * description doesn't need to include {@link #readSchema()}, as Spark 
already knows it.
+   * 
+   * By default this returns the class name of the implementation. Please 
override it to provide a
+   * meaningful description.
+   * 
+   */
+  default String description() {
--- End diff --

I would have expected the default implementation to show both pushed 
filters and the read schema, along with the implementation class name. Read 
schema can be accessed by `readSchema`. Should there also be a way to access 
the pushed filters? `pushedFilters` seems like a good idea to me. (This can be 
added later)


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-29 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237668483
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,65 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.v2.reader.Scan;
+import org.apache.spark.sql.sources.v2.reader.ScanBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * An interface representing a logical structured data set of a data 
source. For example, the
+ * implementation can be a directory on the file system, a topic of Kafka, 
or a table in the
+ * catalog, etc.
+ * 
+ * This interface can mixin the following interfaces to support different 
operations:
+ * 
+ * 
+ *   {@link SupportsBatchRead}: this table can be read in batch 
queries.
+ * 
+ */
+@Evolving
+public interface Table {
+
+  /**
+   * A name to identify this table.
+   * 
+   * By default this returns the class name of this implementation. Please 
override it to provide a
+   * meaningful name, like the database and table name from catalog, or 
the location of files for
+   * this table.
+   * 
+   */
+  default String name() {
--- End diff --

I don't think this should have a default. Implementations should definitely 
implement this.

I think there is a difference between `toString` and `name`. An 
implementation may choose to display `name` when showing a table's string 
representation, but may choose to include extra information to show more about 
the table state, like Iceberg's snapshot ID.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237346499
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
 ---
@@ -22,86 +22,56 @@ import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical
 import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.util.truncatedString
 import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, 
WholeStageCodegenExec}
-import org.apache.spark.sql.execution.streaming.continuous._
-import org.apache.spark.sql.sources.v2.DataSourceV2
 import org.apache.spark.sql.sources.v2.reader._
-import 
org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory,
 ContinuousReadSupport, MicroBatchReadSupport}
 
 /**
- * Physical plan node for scanning data from a data source.
+ * Physical plan node for scanning a batch of data from a data source.
  */
 case class DataSourceV2ScanExec(
 output: Seq[AttributeReference],
-@transient source: DataSourceV2,
-@transient options: Map[String, String],
-@transient pushedFilters: Seq[Expression],
-@transient readSupport: ReadSupport,
-@transient scanConfig: ScanConfig)
-  extends LeafExecNode with DataSourceV2StringFormat with 
ColumnarBatchScan {
+scanDesc: String,
+@transient batch: Batch)
--- End diff --

@rdblue I want to reuse this plan for batch and microbatch. Here this plan 
doesn't take `Scan` but just `Batch`, so that the caller side is flexible to 
decide how to produce batch(es) from a scan.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237346029
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,51 @@
+/*
+ * 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.sources.v2;
--- End diff --

for other reviewers: in the ds v2 community sync, we decided to move data 
source v2 into a new module `sql-api`, and make catalyst depends on it. This 
will be done in a followup.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237346128
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java ---
@@ -0,0 +1,62 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * The base interface for v2 data sources which don't have a real catalog. 
Implementations must
+ * have a public, 0-arg constructor.
+ *
+ * The major responsibility of this interface is to return a {@link Table} 
for read/write.
+ */
+@Evolving
+// TODO: do not extend `DataSourceV2`, after we finish the API refactor 
completely.
+public interface TableProvider extends DataSourceV2 {
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
options.
+   *
+   * @param options the user-specified options that can identify a table, 
e.g. file path, Kafka
+   *topic name, etc. It's an immutable case-insensitive 
string-to-string map.
+   */
+  Table getTable(DataSourceOptions options);
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
schema and options.
+   *
+   * By default this method throws {@link UnsupportedOperationException}, 
implementations should
--- End diff --

added the throw clause.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237342899
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,65 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.v2.reader.Scan;
+import org.apache.spark.sql.sources.v2.reader.ScanBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * An interface representing a logical structured data set of a data 
source. For example, the
+ * implementation can be a directory on the file system, a topic of Kafka, 
or a table in the
+ * catalog, etc.
+ * 
+ * This interface can mixin the following interfaces to support different 
operations:
+ * 
+ * 
+ *   {@link SupportsBatchRead}: this table can be read in batch 
queries.
+ * 
+ */
+@Evolving
+public interface Table {
+
+  /**
+   * A name to identify this table.
+   * 
+   * By default this returns the class name of this implementation. Please 
override it to provide a
+   * meaningful name, like the database and table name from catalog, or 
the location of files for
+   * this table.
+   * 
+   */
+  default String name() {
--- End diff --

Do you think it's better to just ask implementations to override 
`toString`? cc @rdblue 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237179854
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
 ---
@@ -54,27 +53,17 @@ case class DataSourceV2ScanExec(
 Seq(output, source, options).hashCode()
   }
 
-  override def outputPartitioning: physical.Partitioning = readSupport 
match {
+  override def outputPartitioning: physical.Partitioning = scan match {
--- End diff --

If you take my suggesting above to inspect the `Scan` to build the string 
representation of this node, then I think the arguments should be `scan` and 
`output`. Then the batch can be fetched here.

For pushedFilters, I think that they should be fetched from the configured 
scan to build the string representation.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237178976
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
 ---
@@ -23,29 +23,28 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical
 import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
 import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, 
WholeStageCodegenExec}
-import org.apache.spark.sql.execution.streaming.continuous._
 import org.apache.spark.sql.sources.v2.DataSourceV2
 import org.apache.spark.sql.sources.v2.reader._
-import 
org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory,
 ContinuousReadSupport, MicroBatchReadSupport}
 
 /**
- * Physical plan node for scanning data from a data source.
+ * Physical plan node for scanning a batch of data from a data source.
  */
 case class DataSourceV2ScanExec(
 output: Seq[AttributeReference],
 @transient source: DataSourceV2,
 @transient options: Map[String, String],
--- End diff --

With a catalog, there is no expectation that a `source` will be passed. 
This could be a string that identifies either the source or the catalog, for a 
good string representation of the physical plan. This is another area where I 
think `Table.name` would be helpful because the table's identifying information 
is really what should be shown instead of its source or catalog.

For options, these are part of the scan and aren't used to affect the 
behavior of this physical node. I think that means that they shouldn't be part 
of the node's arguments.

I think a good way to solve this problem is to change the pretty string 
format to use `Scan` instead. That has the information that defines what this 
node is doing, like the filters, projection, and options. And being able to 
convert a logical scan to text would be useful across all 3 execution modes.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237176552
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala ---
@@ -38,7 +38,7 @@ import org.apache.spark.sql.execution.datasources.jdbc._
 import 
org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource
 import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
 import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils
-import org.apache.spark.sql.sources.v2.{BatchReadSupportProvider, 
DataSourceOptions, DataSourceV2}
+import org.apache.spark.sql.sources.v2._
--- End diff --

I am using an IDE for this review, but this makes future reviews harder. I 
realize it isn't a major issue, but I think it is a best practice to not use 
wildcard imports.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237176100
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java ---
@@ -0,0 +1,62 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * The base interface for v2 data sources which don't have a real catalog. 
Implementations must
+ * have a public, 0-arg constructor.
+ *
+ * The major responsibility of this interface is to return a {@link Table} 
for read/write.
+ */
+@Evolving
+// TODO: do not extend `DataSourceV2`, after we finish the API refactor 
completely.
+public interface TableProvider extends DataSourceV2 {
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
options.
+   *
+   * @param options the user-specified options that can identify a table, 
e.g. file path, Kafka
+   *topic name, etc. It's an immutable case-insensitive 
string-to-string map.
+   */
+  Table getTable(DataSourceOptions options);
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
schema and options.
+   *
+   * By default this method throws {@link UnsupportedOperationException}, 
implementations should
--- End diff --

Strange, that page links to one with the opposite advice: 
http://www.javapractices.com/topic/TopicAction.do?Id=44

I think that `@throws` is a good idea whenever you want to document an 
exception type as part of the method contract. Since it is expected that this 
method isn't always implemented and may throw this exception, I think you were 
right to document it. And documenting exceptions is best done with `@throws` to 
highlight them in Javadoc.

The page you linked to makes the argument that unchecked exceptions aren't 
part of the method contract and cannot be relied on. But documenting this shows 
that it is part of the contract or expected behavior, so I think docs are 
appropriate.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237172065
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,51 @@
+/*
+ * 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.sources.v2;
--- End diff --

> Can we delay the discussion when we have a PR to add catalog support 
after the refactor?

Yes, that works.

But, can we move `Table` to the `org.apache.spark.sql.catalog.v2` package 
where `TableCatalog` is defined in the other PR? I think `Table` should be 
defined with the catalog API and moving that later would require import changes 
to any file that references `Table`.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237113069
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
 ---
@@ -23,85 +23,55 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical
 import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
 import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, 
WholeStageCodegenExec}
-import org.apache.spark.sql.execution.streaming.continuous._
 import org.apache.spark.sql.sources.v2.DataSourceV2
 import org.apache.spark.sql.sources.v2.reader._
-import 
org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory,
 ContinuousReadSupport, MicroBatchReadSupport}
 
 /**
- * Physical plan node for scanning data from a data source.
+ * Physical plan node for scanning a batch of data from a data source.
  */
 case class DataSourceV2ScanExec(
 output: Seq[AttributeReference],
 @transient source: DataSourceV2,
 @transient options: Map[String, String],
 @transient pushedFilters: Seq[Expression],
-@transient readSupport: ReadSupport,
-@transient scanConfig: ScanConfig)
+@transient batch: Batch)
   extends LeafExecNode with DataSourceV2StringFormat with 
ColumnarBatchScan {
 
   override def simpleString: String = "ScanV2 " + metadataString
 
   // TODO: unify the equal/hashCode implementation for all data source v2 
query plans.
   override def equals(other: Any): Boolean = other match {
-case other: DataSourceV2ScanExec =>
-  output == other.output && readSupport.getClass == 
other.readSupport.getClass &&
-options == other.options
+case other: DataSourceV2ScanExec => this.batch == other.batch
 case _ => false
   }
 
-  override def hashCode(): Int = {
-Seq(output, source, options).hashCode()
-  }
+  override def hashCode(): Int = batch.hashCode()
+
+  private lazy val partitions = batch.planInputPartitions()
+
+  private lazy val readerFactory = batch.createReaderFactory()
 
-  override def outputPartitioning: physical.Partitioning = readSupport 
match {
+  override def outputPartitioning: physical.Partitioning = batch match {
--- End diff --

add back https://github.com/apache/spark/pull/23086#discussion_r236858449


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237111240
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
 ---
@@ -170,15 +157,24 @@ object DataSourceV2Relation {
   }
 
   def create(
-  source: DataSourceV2,
+  provider: TableProvider,
+  table: SupportsBatchRead,
   options: Map[String, String],
   tableIdent: Option[TableIdentifier] = None,
   userSpecifiedSchema: Option[StructType] = None): 
DataSourceV2Relation = {
-val readSupport = source.createReadSupport(options, 
userSpecifiedSchema)
-val output = readSupport.fullSchema().toAttributes
+val output = table.schema().toAttributes
 val ident = tableIdent.orElse(tableFromOptions(options))
 DataSourceV2Relation(
-  source, readSupport, output, options, ident, userSpecifiedSchema)
+  provider, table, output, options, ident, userSpecifiedSchema)
+  }
+
+  def createRelationForWrite(
--- End diff --

done


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237111058
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
 ---
@@ -40,8 +40,8 @@ import org.apache.spark.sql.types.StructType
  * @param userSpecifiedSchema The user-specified schema for this scan.
  */
 case class DataSourceV2Relation(
-source: DataSourceV2,
-readSupport: BatchReadSupport,
+source: TableProvider,
--- End diff --

done


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237110747
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala ---
@@ -38,7 +38,7 @@ import org.apache.spark.sql.execution.datasources.jdbc._
 import 
org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource
 import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
 import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils
-import org.apache.spark.sql.sources.v2.{BatchReadSupportProvider, 
DataSourceOptions, DataSourceV2}
+import org.apache.spark.sql.sources.v2._
--- End diff --

It's the IDE that turns it into wildcard, because it gets too long.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237103452
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
 ---
@@ -54,27 +53,17 @@ case class DataSourceV2ScanExec(
 Seq(output, source, options).hashCode()
   }
 
-  override def outputPartitioning: physical.Partitioning = readSupport 
match {
+  override def outputPartitioning: physical.Partitioning = scan match {
--- End diff --

filter pushdown happens at the planning phase, so the physical plan is the 
only place users can know which filters are pushed. Shall we keep 
`pushedFilters` in the scan node?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237092275
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
 ---
@@ -23,29 +23,28 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical
 import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
 import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, 
WholeStageCodegenExec}
-import org.apache.spark.sql.execution.streaming.continuous._
 import org.apache.spark.sql.sources.v2.DataSourceV2
 import org.apache.spark.sql.sources.v2.reader._
-import 
org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory,
 ContinuousReadSupport, MicroBatchReadSupport}
 
 /**
- * Physical plan node for scanning data from a data source.
+ * Physical plan node for scanning a batch of data from a data source.
  */
 case class DataSourceV2ScanExec(
 output: Seq[AttributeReference],
 @transient source: DataSourceV2,
 @transient options: Map[String, String],
--- End diff --

`source` and `options` are also used to define the string format of this 
plan, as it extends `DataSourceV2StringFormat`.

Maybe we don't need a pretty string format for physical scan node?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237080333
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java ---
@@ -0,0 +1,62 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * The base interface for v2 data sources which don't have a real catalog. 
Implementations must
+ * have a public, 0-arg constructor.
+ *
+ * The major responsibility of this interface is to return a {@link Table} 
for read/write.
+ */
+@Evolving
+// TODO: do not extend `DataSourceV2`, after we finish the API refactor 
completely.
+public interface TableProvider extends DataSourceV2 {
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
options.
+   *
+   * @param options the user-specified options that can identify a table, 
e.g. file path, Kafka
+   *topic name, etc. It's an immutable case-insensitive 
string-to-string map.
+   */
+  Table getTable(DataSourceOptions options);
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
schema and options.
+   *
--- End diff --

thanks for the hint about new paragraph!


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-28 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r237078844
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java ---
@@ -0,0 +1,62 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * The base interface for v2 data sources which don't have a real catalog. 
Implementations must
+ * have a public, 0-arg constructor.
+ *
+ * The major responsibility of this interface is to return a {@link Table} 
for read/write.
+ */
+@Evolving
+// TODO: do not extend `DataSourceV2`, after we finish the API refactor 
completely.
+public interface TableProvider extends DataSourceV2 {
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
options.
+   *
+   * @param options the user-specified options that can identify a table, 
e.g. file path, Kafka
+   *topic name, etc. It's an immutable case-insensitive 
string-to-string map.
+   */
+  Table getTable(DataSourceOptions options);
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
schema and options.
+   *
+   * By default this method throws {@link UnsupportedOperationException}, 
implementations should
--- End diff --

what I learned is that, we should only declare checked exceptions. See 
http://www.javapractices.com/topic/TopicAction.do?Id=171


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236957293
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,51 @@
+/*
+ * 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.sources.v2;
--- End diff --

It's unclear to me what would be the best choice:
1. move data source API to catalyst module
2. move data source related rules to SQL core module
3. define private catalog related APIs in catalyst module and implement 
them in SQL core

Can we delay the discussion when we have a PR to add catalog support after 
the refactor?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236859358
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala 
---
@@ -396,87 +392,66 @@ object SimpleReaderFactory extends 
PartitionReaderFactory {
   }
 }
 
-abstract class SimpleReadSupport extends BatchReadSupport {
-  override def fullSchema(): StructType = new StructType().add("i", 
"int").add("j", "int")
-
-  override def newScanConfigBuilder(): ScanConfigBuilder = {
-NoopScanConfigBuilder(fullSchema())
-  }
+abstract class SimpleBatchTable extends Table with SupportsBatchRead  {
 
-  override def createReaderFactory(config: ScanConfig): 
PartitionReaderFactory = {
-SimpleReaderFactory
-  }
+  override def schema(): StructType = new StructType().add("i", 
"int").add("j", "int")
 }
 
+abstract class SimpleScanBuilder extends ScanBuilder
+  with Batch with Scan {
--- End diff --

I like that the API is flexible enough that `ScanBuilder`, `Scan`, and 
`Batch` can be the same object in simple cases.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236858793
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
 ---
@@ -116,16 +116,20 @@ object DataSourceV2Strategy extends Strategy {
|Output: ${output.mkString(", ")}
  """.stripMargin)
 
-  val scan = DataSourceV2ScanExec(
+  val batch = scan.toBatch
+  val partitions = batch.planInputPartitions()
+  val readerFactory = batch.createReaderFactory()
+  val plan = DataSourceV2ScanExec(
--- End diff --

I mentioned this above, but I think that DataSoruceV2ScanExec only needs to 
be passed `output` and `batch`. That is, unless there is a benefit to calling 
`planInputPartitions` here, like an earlier failure?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236858449
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
 ---
@@ -54,27 +53,17 @@ case class DataSourceV2ScanExec(
 Seq(output, source, options).hashCode()
   }
 
-  override def outputPartitioning: physical.Partitioning = readSupport 
match {
+  override def outputPartitioning: physical.Partitioning = scan match {
--- End diff --

Should `SupportsReportPartitioning` extend `Batch` instead of `Scan`? Then 
this physical node could just be passed the `Batch` and not the `Scan`, 
`PartitionReaderFactory`, and partitions.

In fact, I think that this node only requires `output: 
Seq[AttributeReference], batch: Batch`.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236858107
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
 ---
@@ -23,29 +23,28 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical
 import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
 import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, 
WholeStageCodegenExec}
-import org.apache.spark.sql.execution.streaming.continuous._
 import org.apache.spark.sql.sources.v2.DataSourceV2
 import org.apache.spark.sql.sources.v2.reader._
-import 
org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory,
 ContinuousReadSupport, MicroBatchReadSupport}
 
 /**
- * Physical plan node for scanning data from a data source.
+ * Physical plan node for scanning a batch of data from a data source.
  */
 case class DataSourceV2ScanExec(
 output: Seq[AttributeReference],
 @transient source: DataSourceV2,
 @transient options: Map[String, String],
--- End diff --

Similarly, options were used to create the `Scan` so they don't need to be 
passed here if they are not used in `equals` and `hashCode`.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236857220
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
 ---
@@ -23,29 +23,28 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical
 import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
 import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, 
WholeStageCodegenExec}
-import org.apache.spark.sql.execution.streaming.continuous._
 import org.apache.spark.sql.sources.v2.DataSourceV2
 import org.apache.spark.sql.sources.v2.reader._
-import 
org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory,
 ContinuousReadSupport, MicroBatchReadSupport}
 
 /**
- * Physical plan node for scanning data from a data source.
+ * Physical plan node for scanning a batch of data from a data source.
  */
 case class DataSourceV2ScanExec(
 output: Seq[AttributeReference],
 @transient source: DataSourceV2,
--- End diff --

I think we can remove source by updating `equals` and `hashCode` to check 
just the `Scan`.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236856960
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala
 ---
@@ -23,29 +23,28 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.physical
 import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
 import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, 
WholeStageCodegenExec}
-import org.apache.spark.sql.execution.streaming.continuous._
 import org.apache.spark.sql.sources.v2.DataSourceV2
 import org.apache.spark.sql.sources.v2.reader._
-import 
org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory,
 ContinuousReadSupport, MicroBatchReadSupport}
 
 /**
- * Physical plan node for scanning data from a data source.
+ * Physical plan node for scanning a batch of data from a data source.
  */
 case class DataSourceV2ScanExec(
 output: Seq[AttributeReference],
 @transient source: DataSourceV2,
 @transient options: Map[String, String],
 @transient pushedFilters: Seq[Expression],
-@transient readSupport: ReadSupport,
-@transient scanConfig: ScanConfig)
+@transient scan: Scan,
+@transient partitions: Array[InputPartition],
+@transient readerFactory: PartitionReaderFactory)
   extends LeafExecNode with DataSourceV2StringFormat with 
ColumnarBatchScan {
 
   override def simpleString: String = "ScanV2 " + metadataString
 
   // TODO: unify the equal/hashCode implementation for all data source v2 
query plans.
   override def equals(other: Any): Boolean = other match {
-case other: DataSourceV2ScanExec =>
-  output == other.output && readSupport.getClass == 
other.readSupport.getClass &&
+case other: DataSourceV2StreamingScanExec =>
+  output == other.output && source.getClass == other.source.getClass &&
--- End diff --

Should this implement identity instead of equality? When would two ScanExec 
nodes be equal instead of identical?

Also, I don't think that this equals implementation is correct. First, it 
should not check for the streaming class. Second, it should check whether the 
scan is equal, not whether the options and the source are the same (plus, 
source will be removed).

Unfortunately, implementing true equality (not just identity) must in some 
way rely on a user-supplied class. A scan is the same if it will produce the 
same set of rows and columns in those rows. That means equality depends on the 
filter, projection, and source data (i.e. table). We can use `pushedFilters` 
and `output` for the filter and projection. But checking that the source data 
is the same requires using either the scan's `equals` method (which would also 
satisfy the filter and projection checks) or checking that the partitions are 
the same. Both `Scan` and `InputPartition` implementations are provided by 
sources, so their `equals` methods may not be implemented.

Because this must depend on checking equality of user-supplied objects, I 
think it would be much easier to make this depend only on equality of the 
`Scan`:

```
  override def equals(other: Any): Boolean = other match {
case scanExec: DataSourceV2ScanExec => scanExec.scan == this.scan
  }
```

That may fall back to identity if the user hasn't supplied an equals 
method, but I don't see a way to avoid it.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236852153
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
 ---
@@ -170,15 +157,24 @@ object DataSourceV2Relation {
   }
 
   def create(
-  source: DataSourceV2,
+  provider: TableProvider,
+  table: SupportsBatchRead,
   options: Map[String, String],
   tableIdent: Option[TableIdentifier] = None,
   userSpecifiedSchema: Option[StructType] = None): 
DataSourceV2Relation = {
-val readSupport = source.createReadSupport(options, 
userSpecifiedSchema)
-val output = readSupport.fullSchema().toAttributes
+val output = table.schema().toAttributes
 val ident = tableIdent.orElse(tableFromOptions(options))
 DataSourceV2Relation(
-  source, readSupport, output, options, ident, userSpecifiedSchema)
+  provider, table, output, options, ident, userSpecifiedSchema)
+  }
+
+  def createRelationForWrite(
--- End diff --

Also note that this is temporary until the write side is finished?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236850263
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,51 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.v2.reader.Scan;
+import org.apache.spark.sql.sources.v2.reader.ScanBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * An interface representing a logical structured data set of a data 
source. For example, the
+ * implementation can be a directory on the file system, a topic of Kafka, 
or a table in the
+ * catalog, etc.
+ *
+ * This interface can mixin the following interfaces to support different 
operations:
+ * 
+ *   {@link SupportsBatchRead}: this table can be read in batch 
queries.
+ * 
+ */
+@Evolving
+public interface Table {
--- End diff --

It would be helpful for a `Table` to also expose a name or identifier of 
some kind. The `TableIdentifier` passed into `DataSourceV2Relation` is only 
used in `name` to identify the relation's table. If the name (or location for 
path-based tables) were supplied by the table instead, it would remove the need 
to pass it in the relation.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236849290
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala
 ---
@@ -40,8 +40,8 @@ import org.apache.spark.sql.types.StructType
  * @param userSpecifiedSchema The user-specified schema for this scan.
  */
 case class DataSourceV2Relation(
-source: DataSourceV2,
-readSupport: BatchReadSupport,
+source: TableProvider,
--- End diff --

May want to note that TableProvider will be removed when the write side is 
finished, since it is only used for `createWriteSupport`, which will be exposed 
through `Table`.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236845375
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala ---
@@ -38,7 +38,7 @@ import org.apache.spark.sql.execution.datasources.jdbc._
 import 
org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource
 import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
 import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils
-import org.apache.spark.sql.sources.v2.{BatchReadSupportProvider, 
DataSourceOptions, DataSourceV2}
+import org.apache.spark.sql.sources.v2._
--- End diff --

I do think this one is too nitpicking. If this gets long it should be 
wildcard. Use an IDE for large reviews like this if needed.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236844174
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala ---
@@ -38,7 +38,7 @@ import org.apache.spark.sql.execution.datasources.jdbc._
 import 
org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource
 import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
 import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils
-import org.apache.spark.sql.sources.v2.{BatchReadSupportProvider, 
DataSourceOptions, DataSourceV2}
+import org.apache.spark.sql.sources.v2._
--- End diff --

Nit: using wildcard imports makes it harder to review without an IDE 
because it is more difficult to find out where symbols come from.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236823417
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java ---
@@ -0,0 +1,52 @@
+/*
+ * 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.sources.v2.reader;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.sources.v2.SupportsBatchRead;
+import org.apache.spark.sql.sources.v2.Table;
+
+/**
+ * A logical representation of a data source scan. This interface is used 
to provide logical
+ * information, like what the actual read schema is.
+ *
+ * This logical representation is shared between batch scan, micro-batch 
streaming scan and
+ * continuous streaming scan. Data sources must implement the 
corresponding methods in this
+ * interface, to match what the table promises to support. For example, 
{@link #toBatch()} must be
+ * implemented, if the {@link Table} that creates this {@link Scan} 
implements
+ * {@link SupportsBatchRead}.
+ */
+@Evolving
+public interface Scan {
+
+  /**
+   * Returns the actual schema of this data source scan, which may be 
different from the physical
+   * schema of the underlying storage, as column pruning or other 
optimizations may happen.
+   */
+  StructType readSchema();
+
+  /**
+   * Returns the physical representation of this scan for batch query. By 
default this method throws
+   * exception, data sources must overwrite this method to provide an 
implementation, if the
+   * {@link Table} that creates this scan implements {@link 
SupportsBatchRead}.
+   */
+  default Batch toBatch() {
+throw new UnsupportedOperationException("Do not support batch scan.");
--- End diff --

Nit: text should be "Batch scans are not supported". Starting with "Do not" 
makes the sentence a command.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236820896
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java ---
@@ -0,0 +1,47 @@
+/*
+ * 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.sources.v2.reader;
+
+import org.apache.spark.annotation.Evolving;
+
+/**
+ * A physical representation of a data source scan for batch queries. This 
interface is used to
+ * provide physical information, like how many partitions the scanned data 
has, and how to read
+ * records from the partitions.
+ */
+@Evolving
+public interface Batch {
+
+  /**
+   * Returns a list of {@link InputPartition input partitions}. Each 
{@link InputPartition}
+   * represents a data split that can be processed by one Spark task. The 
number of input
+   * partitions returned here is the same as the number of RDD partitions 
this scan outputs.
+   *
+   * Note that, this may not be a full scan if the data source supports 
optimization like filter
+   * push-down. Implementations should check the status of {@link Scan} 
that creates this batch,
+   * and adjust the resulting {@link InputPartition input partitions}.
--- End diff --

I think this is a little unclear. Implementations do not necessarily check 
the scan. This Batch is likely configured with a filter and is responsible for 
creating splits for that filter.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236820065
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java ---
@@ -0,0 +1,62 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * The base interface for v2 data sources which don't have a real catalog. 
Implementations must
+ * have a public, 0-arg constructor.
+ *
+ * The major responsibility of this interface is to return a {@link Table} 
for read/write.
+ */
+@Evolving
+// TODO: do not extend `DataSourceV2`, after we finish the API refactor 
completely.
+public interface TableProvider extends DataSourceV2 {
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
options.
+   *
+   * @param options the user-specified options that can identify a table, 
e.g. file path, Kafka
+   *topic name, etc. It's an immutable case-insensitive 
string-to-string map.
+   */
+  Table getTable(DataSourceOptions options);
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
schema and options.
+   *
--- End diff --

Minor: Javadoc doesn't automatically parse empty lines as new paragraphs. 
If you want to have one in documentation, then use ``.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236819758
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java ---
@@ -0,0 +1,62 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * The base interface for v2 data sources which don't have a real catalog. 
Implementations must
+ * have a public, 0-arg constructor.
+ *
+ * The major responsibility of this interface is to return a {@link Table} 
for read/write.
+ */
+@Evolving
+// TODO: do not extend `DataSourceV2`, after we finish the API refactor 
completely.
+public interface TableProvider extends DataSourceV2 {
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
options.
+   *
+   * @param options the user-specified options that can identify a table, 
e.g. file path, Kafka
+   *topic name, etc. It's an immutable case-insensitive 
string-to-string map.
+   */
+  Table getTable(DataSourceOptions options);
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
schema and options.
+   *
+   * By default this method throws {@link UnsupportedOperationException}, 
implementations should
--- End diff --

Javadoc would normally also add `@throws` with this information. I agree it 
should be here as well.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236818511
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java ---
@@ -0,0 +1,62 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * The base interface for v2 data sources which don't have a real catalog. 
Implementations must
+ * have a public, 0-arg constructor.
+ *
+ * The major responsibility of this interface is to return a {@link Table} 
for read/write.
+ */
+@InterfaceStability.Evolving
+// TODO: do not extend `DataSourceV2`, after we finish the API refactor 
completely.
+public interface TableProvider extends DataSourceV2 {
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
options.
+   *
+   * @param options the user-specified options that can identify a table, 
e.g. file path, Kafka
+   *topic name, etc. It's an immutable case-insensitive 
string-to-string map.
+   */
+  Table getTable(DataSourceOptions options);
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
schema and options.
+   *
+   * By default this method throws {@link UnsupportedOperationException}, 
implementations should
+   * override this method to handle user-specified schema.
+   *
+   * @param options the user-specified options that can identify a table, 
e.g. file path, Kafka
+   *topic name, etc. It's an immutable case-insensitive 
string-to-string map.
+   * @param schema the user-specified schema.
+   */
+  default Table getTable(DataSourceOptions options, StructType schema) {
--- End diff --

I agree with @cloud-fan. These are slightly different uses.

Here, it is supplying a schema for how to interpret data files. Say you 
have CSV files with columns `id`, `ts`, and `data` and no headers. This tells 
the CSV reader what the columns are and how to convert the data to useful types 
(bigint, timestamp, and string). Column projection will later request those 
columns, maybe just `id` and `data`. If you only passed the projection schema, 
then the `ts` values would be returned for the `data` column.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236816739
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,51 @@
+/*
+ * 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.sources.v2;
--- End diff --

I can understand wanting to keep everything in Catalyst private. That's 
fine with me, but I think that Catalyst does need to be able to interact with 
tables and catalogs that are supplied by users.

For example: Our tables support schema evolution. Specifically, reading 
files that were written before a column was added. When we add a column, Spark 
shouldn't start failing in analysis for an AppendData operation in a scheduled 
job (as it would today). We need to be able to signal to the validation rule 
that the table supports reading files that are missing columns, so that Spark 
can do the right validation and allow writes that used to work to continue.

How would that information -- support for reading missing columns -- be 
communicated to the analyzer?

Also, what about my example above: how will the analyzer load tables using 
a user-supplied catalog if catalyst can't use any user-supplied implementations?

We could move all of the v2 analysis rules, like ResolveRelations, into the 
core module, but it seems to me that this requirement is no longer providing 
value if we have to do that. I think that catalyst is the right place for 
common plans and analysis rules to live because it is the library of common SQL 
components.

Wherever the rules and plans end up, they will need to access to the 
`TableCatalog` API.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-27 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236796331
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,51 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.v2.reader.Scan;
+import org.apache.spark.sql.sources.v2.reader.ScanBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * An interface representing a logical structured data set of a data 
source. For example, the
+ * implementation can be a directory on the file system, a topic of Kafka, 
or a table in the
+ * catalog, etc.
+ *
+ * This interface can mixin the following interfaces to support different 
operations:
+ * 
+ *   {@link SupportsBatchRead}: this table can be read in batch 
queries.
+ * 
+ */
+@Evolving
+public interface Table {
+
+  /**
+   * Returns the schema of this table.
+   */
+  StructType schema();
+
+  /**
+   * Returns a {@link ScanBuilder} which can be used to build a {@link 
Scan} later. Spark will call
+   * this method for each data scanning query.
+   *
+   * The builder can take some query specific information to do operators 
pushdown, and keep these
+   * information in the created {@link Scan}.
+   */
+  ScanBuilder newScanBuilder(DataSourceOptions options);
--- End diff --

Either in a follow-up or you can add the class in this PR. Either way works 
for me.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-26 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236514039
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,51 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.v2.reader.Scan;
+import org.apache.spark.sql.sources.v2.reader.ScanBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * An interface representing a logical structured data set of a data 
source. For example, the
+ * implementation can be a directory on the file system, a topic of Kafka, 
or a table in the
+ * catalog, etc.
+ *
+ * This interface can mixin the following interfaces to support different 
operations:
+ * 
+ *   {@link SupportsBatchRead}: this table can be read in batch 
queries.
+ * 
+ */
+@Evolving
+public interface Table {
+
+  /**
+   * Returns the schema of this table.
+   */
+  StructType schema();
+
+  /**
+   * Returns a {@link ScanBuilder} which can be used to build a {@link 
Scan} later. Spark will call
+   * this method for each data scanning query.
+   *
+   * The builder can take some query specific information to do operators 
pushdown, and keep these
+   * information in the created {@link Scan}.
+   */
+  ScanBuilder newScanBuilder(DataSourceOptions options);
--- End diff --

I agree with it. Since `CaseInsensitiveStringMap` is not in the code base 
yet, shall we do it in the followup?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-26 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236513622
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,51 @@
+/*
+ * 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.sources.v2;
--- End diff --

why does this `Table` API need to be in catalyst? It's not even a plan. We 
can define a table plan interface in catalyst, and implement it in the SQL 
module with this `Table` API.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-26 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236500170
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,51 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.v2.reader.Scan;
+import org.apache.spark.sql.sources.v2.reader.ScanBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * An interface representing a logical structured data set of a data 
source. For example, the
+ * implementation can be a directory on the file system, a topic of Kafka, 
or a table in the
+ * catalog, etc.
+ *
+ * This interface can mixin the following interfaces to support different 
operations:
+ * 
+ *   {@link SupportsBatchRead}: this table can be read in batch 
queries.
+ * 
+ */
+@Evolving
+public interface Table {
+
+  /**
+   * Returns the schema of this table.
+   */
+  StructType schema();
+
+  /**
+   * Returns a {@link ScanBuilder} which can be used to build a {@link 
Scan} later. Spark will call
+   * this method for each data scanning query.
+   *
+   * The builder can take some query specific information to do operators 
pushdown, and keep these
+   * information in the created {@link Scan}.
+   */
+  ScanBuilder newScanBuilder(DataSourceOptions options);
--- End diff --

Makes sense to me - `DataSourceOptions` was carrying along identifiers that 
really belong to a table identifier and that should be interpreted at the 
catalog level, not the data read level. In other words the implementation of 
this `Table` should already know _what_ locations to look up (e.g. "files 
comprising dataset D"), now it's a matter of _how_ (e.g. pushdown, filter 
predicates).


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-26 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236492408
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,51 @@
+/*
+ * 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.sources.v2;
--- End diff --

Everything in catalyst is considered private (although public visibility 
for debugging) and it's best to stay that way.



---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-26 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236491385
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,51 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.Evolving;
+import org.apache.spark.sql.sources.v2.reader.Scan;
+import org.apache.spark.sql.sources.v2.reader.ScanBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * An interface representing a logical structured data set of a data 
source. For example, the
+ * implementation can be a directory on the file system, a topic of Kafka, 
or a table in the
+ * catalog, etc.
+ *
+ * This interface can mixin the following interfaces to support different 
operations:
+ * 
+ *   {@link SupportsBatchRead}: this table can be read in batch 
queries.
+ * 
+ */
+@Evolving
+public interface Table {
+
+  /**
+   * Returns the schema of this table.
+   */
+  StructType schema();
+
+  /**
+   * Returns a {@link ScanBuilder} which can be used to build a {@link 
Scan} later. Spark will call
+   * this method for each data scanning query.
+   *
+   * The builder can take some query specific information to do operators 
pushdown, and keep these
+   * information in the created {@link Scan}.
+   */
+  ScanBuilder newScanBuilder(DataSourceOptions options);
--- End diff --

`DataSourceOptions` isn't simply a map for two main reasons that I can 
tell: first, it forces options to be case insensitive, and second, it exposes 
helper methods to identify tables, like `tableName`, `databaseName`, and 
`paths`. In the new abstraction, the second use of `DataSourceOptions` is no 
longer needed. The table is already instantiated by the time that this is 
called.

We should to reconsider `DataSourceOptions`. The `tableName` methods aren't 
needed and we also no longer need to forward properties from the session config 
because the way tables are configured has changed (catalogs handle that). I 
think we should remove this class and instead use the more direct 
implementation, `CaseInsensitiveStringMap` from #21306. The behavior of that 
class is obvious from its name and it would be shared between the v2 APIs, both 
catalog and data source.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-26 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236490800
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,51 @@
+/*
+ * 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.sources.v2;
--- End diff --

Moving this to the Catalyst package would set a precedent for 
user-overridable behavior to live in the catalyst project. I'm not aware of 
anything in the Catalyst package being considered as public API right now. Are 
we allowed to start such a convention at this juncture?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-26 Thread rdblue
Github user rdblue commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r236487464
  
--- Diff: sql/core/src/main/java/org/apache/spark/sql/sources/v2/Table.java 
---
@@ -0,0 +1,51 @@
+/*
+ * 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.sources.v2;
--- End diff --

#21306 (TableCatalog support) adds this class as 
`org.apache.spark.sql.catalog.v2.Table` in the `spark-catalyst` module. I think 
it needs to be in the catalyst module and should probably be in the 
`o.a.s.sql.catalog.v2` package as well.

The important one is moving this to the catalyst module. The analyzer is in 
catalyst and all of the v2 logical plans and analysis rules will be in catalyst 
as well, because we are standardizing behavior. The standard validation rules 
should be in catalyst, not in a source-specific or hive-specific package in the 
sql-core or hive modules.

Because the logical plans and validation rules are in the catalyst package, 
the `TableCatalog` API needs to be there as well. For example, when a [catalog 
table identifier](https://github.com/apache/spark/pull/21978) is resolved for a 
read query, one of the results is a `TableCatalog` instance for the catalog 
portion of the identifier. That catalog is used to load the v2 table, which is 
then wrapped in a v2 relation for further analysis. Similarly, the write path 
should also validate that the catalog exists during analysis by loading it, and 
would then pass the catalog in a v2 logical plan for `CreateTable` or 
`CreateTableAsSelect`.

I also think that it makes sense to use the 
`org.apache.spark.sql.catalog.v2` package for `Table` because `Table` is more 
closely tied to the `TableCatalog` API than to the data source API. The link to 
DSv2 is that `Table` carries `newScanBuilder`, but the rest of the methods 
exposed by `Table` are for catalog functions, like inspecting a table's 
partitioning or table properties.

Moving this class would make adding `TableCatalog` less intrusive.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-21 Thread jose-torres
Github user jose-torres commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r235470285
  
--- Diff: project/MimaExcludes.scala ---
@@ -149,7 +149,8 @@ object MimaExcludes {
 
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader"),
 
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.writer.DataSourceWriter"),
 
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.writer.DataWriterFactory.createWriter"),
-
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter")
+
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter"),
--- End diff --

This list of exclusions is getting kinda silly. Is there some way to just 
completely exclude this package from compatibility checks until we've 
stabilized it?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r234855810
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java ---
@@ -0,0 +1,47 @@
+/*
+ * 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.sources.v2.reader;
+
+import org.apache.spark.annotation.InterfaceStability;
+
+/**
+ * A physical representation of a data source scan for batch queries. This 
interface is used to
+ * provide physical information, like how many partitions the scanned data 
has, and how to read
+ * records from the partitions.
+ */
+@InterfaceStability.Evolving
+public interface Batch {
--- End diff --

I don't have a strong preference. I feel it's a little more clear to 
distinguish between scan and batch


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r234844186
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java ---
@@ -0,0 +1,62 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * The base interface for v2 data sources which don't have a real catalog. 
Implementations must
+ * have a public, 0-arg constructor.
+ *
+ * The major responsibility of this interface is to return a {@link Table} 
for read/write.
+ */
+@InterfaceStability.Evolving
+// TODO: do not extend `DataSourceV2`, after we finish the API refactor 
completely.
+public interface TableProvider extends DataSourceV2 {
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
options.
+   *
+   * @param options the user-specified options that can identify a table, 
e.g. file path, Kafka
+   *topic name, etc. It's an immutable case-insensitive 
string-to-string map.
+   */
+  Table getTable(DataSourceOptions options);
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
schema and options.
+   *
+   * By default this method throws {@link UnsupportedOperationException}, 
implementations should
+   * override this method to handle user-specified schema.
+   *
+   * @param options the user-specified options that can identify a table, 
e.g. file path, Kafka
+   *topic name, etc. It's an immutable case-insensitive 
string-to-string map.
+   * @param schema the user-specified schema.
+   */
+  default Table getTable(DataSourceOptions options, StructType schema) {
--- End diff --

It's a different thing. Think about you are reading a parquet file, and you 
know exactly what its physical schema is, and you don't want Spark to waste a 
job to infer the schema. Then you can specify the schema when reading.

Next, Spark will analyze the query, and figure out what the required schema 
is. This step is automatic and driven by Spark.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-19 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r234793177
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java ---
@@ -0,0 +1,62 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * The base interface for v2 data sources which don't have a real catalog. 
Implementations must
+ * have a public, 0-arg constructor.
+ *
+ * The major responsibility of this interface is to return a {@link Table} 
for read/write.
+ */
+@InterfaceStability.Evolving
+// TODO: do not extend `DataSourceV2`, after we finish the API refactor 
completely.
+public interface TableProvider extends DataSourceV2 {
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
options.
+   *
+   * @param options the user-specified options that can identify a table, 
e.g. file path, Kafka
+   *topic name, etc. It's an immutable case-insensitive 
string-to-string map.
+   */
+  Table getTable(DataSourceOptions options);
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
schema and options.
+   *
+   * By default this method throws {@link UnsupportedOperationException}, 
implementations should
+   * override this method to handle user-specified schema.
+   *
+   * @param options the user-specified options that can identify a table, 
e.g. file path, Kafka
+   *topic name, etc. It's an immutable case-insensitive 
string-to-string map.
+   * @param schema the user-specified schema.
+   */
+  default Table getTable(DataSourceOptions options, StructType schema) {
--- End diff --

Basically just saying we should just push down this requested schema into 
the `ScanBuilder`.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-19 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r234736735
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java ---
@@ -0,0 +1,62 @@
+/*
+ * 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.sources.v2;
+
+import org.apache.spark.annotation.InterfaceStability;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * The base interface for v2 data sources which don't have a real catalog. 
Implementations must
+ * have a public, 0-arg constructor.
+ *
+ * The major responsibility of this interface is to return a {@link Table} 
for read/write.
+ */
+@InterfaceStability.Evolving
+// TODO: do not extend `DataSourceV2`, after we finish the API refactor 
completely.
+public interface TableProvider extends DataSourceV2 {
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
options.
+   *
+   * @param options the user-specified options that can identify a table, 
e.g. file path, Kafka
+   *topic name, etc. It's an immutable case-insensitive 
string-to-string map.
+   */
+  Table getTable(DataSourceOptions options);
+
+  /**
+   * Return a {@link Table} instance to do read/write with user-specified 
schema and options.
+   *
+   * By default this method throws {@link UnsupportedOperationException}, 
implementations should
+   * override this method to handle user-specified schema.
+   *
+   * @param options the user-specified options that can identify a table, 
e.g. file path, Kafka
+   *topic name, etc. It's an immutable case-insensitive 
string-to-string map.
+   * @param schema the user-specified schema.
+   */
+  default Table getTable(DataSourceOptions options, StructType schema) {
--- End diff --

I know that this is from prior DataSourceV2 semantics, but what's the 
difference between providing the `schema` here and the column pruning aspect of 
`ScanBuilder`?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-19 Thread mccheah
Github user mccheah commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r234736935
  
--- Diff: 
sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java ---
@@ -0,0 +1,47 @@
+/*
+ * 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.sources.v2.reader;
+
+import org.apache.spark.annotation.InterfaceStability;
+
+/**
+ * A physical representation of a data source scan for batch queries. This 
interface is used to
+ * provide physical information, like how many partitions the scanned data 
has, and how to read
+ * records from the partitions.
+ */
+@InterfaceStability.Evolving
+public interface Batch {
--- End diff --

`BatchScan`, perhaps?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-19 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/23086#discussion_r234650248
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StreamingScanExec.scala
 ---
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.physical
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, 
WholeStageCodegenExec}
+import org.apache.spark.sql.execution.streaming.continuous._
+import org.apache.spark.sql.sources.v2.DataSourceV2
+import org.apache.spark.sql.sources.v2.reader._
+import 
org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory,
 ContinuousReadSupport, MicroBatchReadSupport}
+
+/**
+ * Physical plan node for scanning data from a data source.
+ */
+// TODO: micro-batch should be handled by `DataSourceV2ScanExec`, after we 
finish the API refactor
+// completely.
+case class DataSourceV2StreamingScanExec(
--- End diff --

I have to use two physical nodes, since batch and streaming have different 
APIs now.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #23086: [SPARK-25528][SQL] data source v2 API refactor (b...

2018-11-19 Thread cloud-fan
GitHub user cloud-fan opened a pull request:

https://github.com/apache/spark/pull/23086

[SPARK-25528][SQL] data source v2 API refactor (batch read)

## What changes were proposed in this pull request?

This is the first step of the data source v2 API refactor 
[proposal](https://docs.google.com/document/d/1uUmKCpWLdh9vHxP7AWJ9EgbwB_U6T3EJYNjhISGmiQg/edit?usp=sharing)

It adds the new API for batch read, without removing the old APIs, as they 
are still needed for streaming sources.

More concretely, it adds
1. `TableProvider`, works like an anonymous catalog
2. `Table`, represents a structured data set.
3. `ScanBuilder` and `Scan`, a logical represents of data source scan
4. `Batch`, a physical representation of data source batch scan.

## How was this patch tested?

existing tests


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/cloud-fan/spark refactor-batch

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/23086.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #23086


commit f06b5c58b1a890d425abd575fa6f4c40da7c4b3d
Author: Wenchen Fan 
Date:   2018-11-19T11:05:07Z

data source v2 API refactor (batch read)




---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org