From 5f4c26ba2d0631f46e0a389efa9b0e6ac19e09f9 Mon Sep 17 00:00:00 2001 From: shivsood Date: Mon, 15 Jul 2019 21:30:35 -0700 Subject: [PATCH 01/24] scaffolding : basic read/write with jdbc connector --- examples/src/main/python/sql/datasourcev2.py | 20 +++++ ...pache.spark.sql.sources.DataSourceRegister | 1 + .../v2/jdbc/DBPartitionReader.scala | 76 ++++++++++++++++++ .../v2/jdbc/DBPartitionReaderFactory.scala | 32 ++++++++ .../datasources/v2/jdbc/DBTable.scala | 78 +++++++++++++++++++ .../datasources/v2/jdbc/DBTableScan.scala | 56 +++++++++++++ .../datasources/v2/jdbc/JDBCBatchWrite.scala | 42 ++++++++++ .../v2/jdbc/JDBCDataSourceV2.scala | 43 ++++++++++ .../datasources/v2/jdbc/JDBCDataWriter.scala | 56 +++++++++++++ .../v2/jdbc/JDBCDataWriterFactory.scala | 32 ++++++++ .../datasources/v2/jdbc/JDBCScanBuilder.scala | 55 +++++++++++++ .../v2/jdbc/JDBCWriteBuilder.scala | 40 ++++++++++ 12 files changed, 531 insertions(+) create mode 100644 examples/src/main/python/sql/datasourcev2.py create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala diff --git a/examples/src/main/python/sql/datasourcev2.py b/examples/src/main/python/sql/datasourcev2.py new file mode 100644 index 000000000000..392d17ae00ff --- /dev/null +++ b/examples/src/main/python/sql/datasourcev2.py @@ -0,0 +1,20 @@ +from pyspark.sql import SparkSession +from pyspark.sql.functions import col + +spark = SparkSession\ + .builder\ + .appName("DV2Test")\ + .getOrCreate() + + +df = spark.read.format("csv").options(header='true').load("/home/shivsood/myspark_scripts/text.csv") +df.show(5) + +#Read +df_jdbc = spark.read.options(header='true').format("jdbcv2").load() +df_jdbc.show(3) + + +#Write +#df.filter( col("rollnum") == "38" ).write.format("jdbcv2").save() + diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index c0b8b270bab1..5ce0de75ec46 100644 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -9,3 +9,4 @@ org.apache.spark.sql.execution.streaming.ConsoleSinkProvider org.apache.spark.sql.execution.streaming.sources.RateStreamProvider org.apache.spark.sql.execution.streaming.sources.TextSocketSourceProvider org.apache.spark.sql.execution.datasources.binaryfile.BinaryFileFormat +org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCDataSourceV2 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala new file mode 100644 index 000000000000..9ce41711ca9b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala @@ -0,0 +1,76 @@ +/* + * 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.jdbc + +import java.io.IOException + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.unsafe.types.UTF8String + +class DBPartitionReader(schema : StructType) extends PartitionReader[InternalRow] with Logging { + + var dummyRows = 0 + + @throws[IOException] + def next(): Boolean = { + + logInfo("***dsv2-flows*** next() called") + + if(dummyRows <2) { + dummyRows = dummyRows + 1 + true + } else { + false + } + } + + def get: InternalRow = { + + logInfo("***dsv2-flows*** get() called for row " + dummyRows) + + // Value for row1 + var v_name = "shiv" + var v_rollnum = "38" + var v_occupation = "worker" + + if(dummyRows == 2) { + // Values for row2 + v_name = "someone" + v_rollnum = "39" + v_occupation = "dontknow" + } + + val values = schema.map(_.name).map { + case "name" => UTF8String.fromString(v_name) + case "rollnum" => UTF8String.fromString(v_rollnum) + case "occupation" => UTF8String.fromString(v_occupation) + case _ => UTF8String.fromString("anything") + } + + InternalRow.fromSeq(values) + } + + @throws[IOException] + override def close(): Unit = { + + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala new file mode 100644 index 000000000000..3c3d3a43e14a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala @@ -0,0 +1,32 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.types.StructType + +class DBPartitionReaderFactory(schema : StructType) extends PartitionReaderFactory with Logging{ + + def createReader(partition: InputPartition): PartitionReader[InternalRow] = { + logInfo("***dsv2-flows*** createReader called") + new DBPartitionReader(schema) + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala new file mode 100644 index 000000000000..478ceb9c28cf --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -0,0 +1,78 @@ +/* + * 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.jdbc + +import scala.collection.JavaConverters._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.v2.FileTable +import org.apache.spark.sql.execution.datasources.v2.csv.CSVWriteBuilder +import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.sources.v2.TableCapability.{BATCH_READ, BATCH_WRITE, TRUNCATE} +import org.apache.spark.sql.sources.v2.reader.ScanBuilder +import org.apache.spark.sql.sources.v2.writer.WriteBuilder +import org.apache.spark.sql.types.{DataType, IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class DBTable (sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + userSpecifiedSchema: Option[StructType]) + extends Table with SupportsWrite with SupportsRead with Logging{ + + + override def name: String = { + // TODO - Should come from user options + + logInfo("***dsv2-flows*** name called") + + "mysqltable" + } + + def schema: StructType = { + // TODO - Remove hardcoded schema + logInfo("***dsv2-flows*** schema called") + StructType(Seq( + StructField("name", StringType, true), + StructField("rollnum", StringType, true), + StructField("occupation", StringType, true))) + } + + override def capabilities: java.util.Set[TableCapability] = DBTable.CAPABILITIES + + def supportsDataType(dataType: DataType): Boolean = true + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + logInfo("***dsv2-flows*** newWriteBuilder called") + new JDBCWriteBuilder() + } + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + logInfo("***dsv2-flows*** newScanBuilder called") + new JDBCScanBuilder() + } + +} + +object DBTable { + private val CAPABILITIES = Set(BATCH_WRITE, BATCH_READ, TRUNCATE).asJava +} + + + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala new file mode 100644 index 000000000000..6b876dfc9429 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala @@ -0,0 +1,56 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReaderFactory, Scan} +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +class DBTableScan extends Scan with Batch with Logging { + + val table_schema = StructType(Seq( + StructField("name", StringType, true), + StructField("rollnum", StringType, true), + StructField("occupation", StringType, true))) + + def readSchema: StructType = { + logInfo("***dsv2-flows*** readSchema called") + table_schema + + } + + override def toBatch() : Batch = { + this + } + + def planInputPartitions: Array[InputPartition] = { + Array(PartitionScheme) + } + + def createReaderFactory: PartitionReaderFactory = { + + logInfo("***dsv2-flows*** createReaderFactory called") + new DBPartitionReaderFactory(table_schema) + + } + +} + +object PartitionScheme extends InputPartition { + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala new file mode 100644 index 000000000000..e4704b8bc737 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala @@ -0,0 +1,42 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, WriterCommitMessage} + +class JDBCBatchWrite extends BatchWrite with Logging{ + + def createBatchWriterFactory: DataWriterFactory = { + logInfo("***dsv2-flows*** createBatchWriterFactory called" ) + new JDBCDataWriterFactory() + } + + + def commit(messages: Array[WriterCommitMessage]): Unit = { + + logInfo("***dsv2-flows*** commit called with message... " ) + + } + + def abort(messages: Array[WriterCommitMessage]): Unit = { + logInfo("***dsv2-flows*** abort called with message... " ) + + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala new file mode 100644 index 000000000000..23ccc65ea082 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala @@ -0,0 +1,43 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.sources.v2.{Table, TableProvider} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class JDBCDataSourceV2 extends TableProvider with DataSourceRegister with Logging{ + + override def shortName(): String = { + logInfo("***dsv2-flows*** shortName - return connector name") + "jdbcv2" + } + + override def getTable(options: CaseInsensitiveStringMap): Table = { + logInfo("***dsv2-flows*** getTable called") + DBTable(SparkSession.active, options, None) + } + + override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { + logInfo("***dsv2-flows*** getTable called with schema") + DBTable(SparkSession.active, options, Some(schema)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala new file mode 100644 index 000000000000..91caf1a4058b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala @@ -0,0 +1,56 @@ +/* + * 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.jdbc + +import java.io.IOException + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage} +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +class JDBCDataWriter extends DataWriter[InternalRow] with Logging{ + + @throws[IOException] + def write(record: InternalRow): Unit = { + logInfo("***dsv2-flows*** write " ) + + val schema = StructType(Seq( + StructField("name", StringType, true), + StructField("rollnum", StringType, true), + StructField("occupation", StringType, true))) + + val ret = record.toSeq(schema) + logInfo("***dsv2-flows*** write " + ret.mkString(":") ) + } + + @throws[IOException] + def commit: WriterCommitMessage = { + logInfo("***dsv2-flows*** commit called " ) + JDBCWriterCommitMessage + } + + @throws[IOException] + def abort(): Unit = { + logInfo("***dsv2-flows*** abort called " ) + } +} + +object JDBCWriterCommitMessage extends WriterCommitMessage { + val commitMessage: String = "committed" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala new file mode 100644 index 000000000000..f8f99716228f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala @@ -0,0 +1,32 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory} + +class JDBCDataWriterFactory extends DataWriterFactory with Logging{ + + def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = { + logInfo("***dsv2-flows*** createWriter called " ) + + new JDBCDataWriter() + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala new file mode 100644 index 000000000000..ac927b70dde7 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala @@ -0,0 +1,55 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.v2.SupportsRead +import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.types.StructType + +class JDBCScanBuilder extends ScanBuilder with + SupportsPushDownFilters with SupportsPushDownRequiredColumns + with Logging { + + var specifiedFilters: Array[Filter] = Array.empty + + + def build: Scan = { + logInfo("***dsv2-flows*** Scan called") + new DBTableScan() + + } + + def pushFilters(filters: Array[Filter]): Array[Filter] = { + logInfo("***dsv2-flows*** PushDown filters called") + specifiedFilters = filters + filters + } + + def pruneColumns(requiredSchema: StructType): Unit = { + logInfo("***dsv2-flows*** pruneColumns called") + + } + + def pushedFilters: Array[Filter] = { + logInfo("***dsv2-flows*** pushedFilters called") + specifiedFilters + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala new file mode 100644 index 000000000000..bb50a965edaa --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala @@ -0,0 +1,40 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, WriteBuilder} +import org.apache.spark.sql.types.StructType + +class JDBCWriteBuilder extends WriteBuilder with Logging{ + + override def withQueryId(queryId: String): WriteBuilder = { + logInfo("***dsv2-flows*** withQueryId called with queryId" + queryId) + this + } + + override def withInputDataSchema(schema: StructType): WriteBuilder = { + logInfo("***dsv2-flows*** withInputDataSchema called with schema" + schema.printTreeString()) + this + } + + override def buildForBatch : BatchWrite = { + logInfo("***dsv2-flows*** BatchWrite called") + new JDBCBatchWrite() + } +} From e238fc0d521d0cf41342e0b7fbf2a0f471fef442 Mon Sep 17 00:00:00 2001 From: shivsood Date: Fri, 19 Jul 2019 15:45:39 -0700 Subject: [PATCH 02/24] first draft implemnentation of write(append) flow with datasourcev2. E2E test case added in MssQLServerIntegrationSuite --- .../jdbc/MsSqlServerIntegrationSuite.scala | 24 ++++++ .../datasources/jdbc/JdbcUtils.scala | 78 +++++++++++++++++ .../v2/jdbc/DBPartitionReaderFactory.scala | 1 - .../datasources/v2/jdbc/DBTable.scala | 52 ++++++------ .../datasources/v2/jdbc/JDBCBatchWrite.scala | 16 ++-- .../v2/jdbc/JDBCDataSourceV2.scala | 1 - .../datasources/v2/jdbc/JDBCDataWriter.scala | 22 ++--- .../v2/jdbc/JDBCDataWriterFactory.scala | 19 +++-- .../v2/jdbc/JDBCWriteBuilder.scala | 83 +++++++++++++++++-- .../execution/datasources/v2/jdbc/Utils.scala | 58 +++++++++++++ 10 files changed, 295 insertions(+), 59 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index efd7ca74c796..edc3cf0761e6 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -99,6 +99,30 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { |'the', 'lazy', |'dog') """.stripMargin).executeUpdate() + conn.prepareStatement( + """ + |CREATE TABLE strings_numbers ( + |i NVarChar(10), + |j INT, + |k NVarChar(20)) + """.stripMargin).executeUpdate() + conn.prepareStatement( + """ + |INSERT INTO strings_numbers VALUES ( + |'string',38, + |'big string') + """.stripMargin).executeUpdate() + } + + test("JDBCV2 write test") { + // Read 1 row using JDBC. Write(append) this row using jdbcv2. + val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() + df1.show(10) + assert(df1.count == 1) + df1.write.format("jdbcv2").mode("append").option("url",jdbcUrl).option("dbtable", "strings_numbers").save() + val df2 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() + df2.show(10) + assert(df2.count == 2) } test("Basic test") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 86a27b5afc25..63f62b31f89e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -870,4 +870,82 @@ object JdbcUtils extends Logging { statement.close() } } + + /** + * Inserts row into the database represented by the connection. + * + * @param conn - JDBCConnection to the given database. + * @param record - InternalRow to be inserted + * @throws IllegalArgumentException if the schema contains an unsupported type. + * Additions to JDBCUtils to support DSV2 JDBC adapter. + * TODO : Refactoring to remove makeIRSetter and use makeSetter with InternalRows + * TODO : Reactor to remove v2 imports/types from this pure DSV1 file. + * TODO : Support for Transactions, Isolation levels + * TODO : Just add to statement here. Commit should be outside of this function. + */ + def saveRow(conn: Connection, record: InternalRow, + options : JdbcOptionsInWrite, schema : StructType) : Unit = { + val dialect = JdbcDialects.get(options.url) + val rddSchema = JdbcUtils.getSchemaOption(conn, options) + val passedSchema = schema + + val insertStatement = JdbcUtils.getInsertStatement(options.table, + schema, rddSchema, true, dialect) + val stmt = conn.prepareStatement(insertStatement) + logInfo("***dsv2-flows*** insertStatement is $insertStatement") + val setters = schema.fields.map(f => makeIRSetter(conn, dialect, f.dataType)) + val nullTypes = schema.fields.map(f => getJdbcType(f.dataType, dialect).jdbcNullType) + val numFields = schema.fields.length + + var colNum = 0 + while (colNum < numFields) { + logInfo("***dsv2-flows*** stmt prep field " + colNum) + if (record.isNullAt(colNum)) { + stmt.setNull(colNum + 1, nullTypes(colNum)) + } else { + setters(colNum).apply(stmt, record, colNum) + } + colNum = colNum + 1 + } + stmt.execute() + } + + private type JDBCValueSetterIR = (PreparedStatement, InternalRow, Int) => Unit + + private def makeIRSetter(conn: Connection, dialect: JdbcDialect, + dataType: DataType): JDBCValueSetterIR = dataType match { + case IntegerType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setInt(pos + 1, row.getInt(pos)) + + case LongType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setLong(pos + 1, row.getLong(pos)) + + case DoubleType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setDouble(pos + 1, row.getDouble(pos)) + + case FloatType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setFloat(pos + 1, row.getFloat(pos)) + + case ShortType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setShort(pos + 1, row.getShort(pos)) + + case ByteType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setByte(pos + 1, row.getByte(pos)) + + case BooleanType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setBoolean(pos + 1, row.getBoolean(pos)) + + case StringType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setString(pos + 1, row.getString(pos)) + case _ => + throw new IllegalArgumentException(s"Not supported ${dataType.catalogString}") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala index 3c3d3a43e14a..03d86fa18229 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala @@ -28,5 +28,4 @@ class DBPartitionReaderFactory(schema : StructType) extends PartitionReaderFacto logInfo("***dsv2-flows*** createReader called") new DBPartitionReader(schema) } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala index 478ceb9c28cf..cea507454221 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -17,62 +17,60 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc +import java.sql.Connection + import scala.collection.JavaConverters._ import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.execution.datasources.v2.csv.CSVWriteBuilder +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions.JDBC_TABLE_NAME +import org.apache.spark.sql.execution.streaming.MetadataLogFileIndex import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.sources.v2.TableCapability.{BATCH_READ, BATCH_WRITE, TRUNCATE} +import org.apache.spark.sql.sources.v2.TableCapability.{BATCH_READ, BATCH_WRITE, OVERWRITE_BY_FILTER, TRUNCATE} import org.apache.spark.sql.sources.v2.reader.ScanBuilder import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types.{DataType, IntegerType, StringType, StructField, StructType} -import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql.util.{CaseInsensitiveStringMap, SchemaUtils} case class DBTable (sparkSession: SparkSession, - options: CaseInsensitiveStringMap, - userSpecifiedSchema: Option[StructType]) + options: CaseInsensitiveStringMap, + userSchema: Option[StructType]) extends Table with SupportsWrite with SupportsRead with Logging{ + private val userOptions = new JDBCOptions(options.asScala.toMap) + private val tableName = userOptions.parameters(JDBC_TABLE_NAME) + private val conn : Connection = JdbcUtils.createConnectionFactory(userOptions)() override def name: String = { - // TODO - Should come from user options - - logInfo("***dsv2-flows*** name called") - - "mysqltable" + logInfo("***dsv2-flows*** name called. Table name is " + tableName) + tableName } - def schema: StructType = { - // TODO - Remove hardcoded schema + override def schema: StructType = { + // TODO - check why a schema request? What if no table exists and + // no userSpecifiedSchema logInfo("***dsv2-flows*** schema called") - StructType(Seq( - StructField("name", StringType, true), - StructField("rollnum", StringType, true), - StructField("occupation", StringType, true))) + val schemaInDB = JdbcUtils.getSchemaOption(conn, userOptions) + Utils.logSchema("schema from DB", schemaInDB) + schemaInDB.getOrElse(StructType(Nil)) } override def capabilities: java.util.Set[TableCapability] = DBTable.CAPABILITIES - def supportsDataType(dataType: DataType): Boolean = true - override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { logInfo("***dsv2-flows*** newWriteBuilder called") - new JDBCWriteBuilder() + Utils.logSchema("Schema passed to DBTable", userSchema) + new JDBCWriteBuilder( + new JdbcOptionsInWrite(options.asScala.toMap), userSchema) } override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { logInfo("***dsv2-flows*** newScanBuilder called") new JDBCScanBuilder() } - } object DBTable { - private val CAPABILITIES = Set(BATCH_WRITE, BATCH_READ, TRUNCATE).asJava + private val CAPABILITIES = Set(BATCH_READ, BATCH_WRITE, TRUNCATE, OVERWRITE_BY_FILTER).asJava } - - - diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala index e4704b8bc737..952e05eb7530 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala @@ -18,25 +18,23 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.datasources.jdbc.JdbcOptionsInWrite import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.types.StructType -class JDBCBatchWrite extends BatchWrite with Logging{ +class JDBCBatchWrite(options: JdbcOptionsInWrite, fwPassedSchema: StructType) + extends BatchWrite with Logging{ def createBatchWriterFactory: DataWriterFactory = { logInfo("***dsv2-flows*** createBatchWriterFactory called" ) - new JDBCDataWriterFactory() + new JDBCDataWriterFactory(options, fwPassedSchema) } - - def commit(messages: Array[WriterCommitMessage]): Unit = { - - logInfo("***dsv2-flows*** commit called with message... " ) - + logInfo("***dsv2-flows*** commit called with $messages.length messages" + + s"with value as $messages.mkString(':')") } def abort(messages: Array[WriterCommitMessage]): Unit = { logInfo("***dsv2-flows*** abort called with message... " ) - } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala index 23ccc65ea082..aec0940b5bad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap class JDBCDataSourceV2 extends TableProvider with DataSourceRegister with Logging{ - override def shortName(): String = { logInfo("***dsv2-flows*** shortName - return connector name") "jdbcv2" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala index 91caf1a4058b..016921ca25c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala @@ -18,25 +18,24 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import java.io.IOException +import java.sql.{Connection, PreparedStatement} import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils._ +import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage} -import org.apache.spark.sql.types.{StringType, StructField, StructType} - -class JDBCDataWriter extends DataWriter[InternalRow] with Logging{ +import org.apache.spark.sql.types._ +class JDBCDataWriter(options: JdbcOptionsInWrite, + jdbcConn: Connection, schema: StructType) + extends DataWriter[InternalRow] with Logging{ @throws[IOException] def write(record: InternalRow): Unit = { logInfo("***dsv2-flows*** write " ) - - val schema = StructType(Seq( - StructField("name", StringType, true), - StructField("rollnum", StringType, true), - StructField("occupation", StringType, true))) - - val ret = record.toSeq(schema) - logInfo("***dsv2-flows*** write " + ret.mkString(":") ) + JdbcUtils.saveRow(jdbcConn, record, options, schema) } @throws[IOException] @@ -49,6 +48,7 @@ class JDBCDataWriter extends DataWriter[InternalRow] with Logging{ def abort(): Unit = { logInfo("***dsv2-flows*** abort called " ) } + } object JDBCWriterCommitMessage extends WriterCommitMessage { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala index f8f99716228f..db48840fba38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala @@ -17,16 +17,25 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc +import java.sql.Connection + import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory} +import org.apache.spark.sql.types.StructType -class JDBCDataWriterFactory extends DataWriterFactory with Logging{ - +/* Writer factory that's serialized and send to executors + */ +class JDBCDataWriterFactory(options: JdbcOptionsInWrite, schema: StructType) extends + DataWriterFactory with Logging{ def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = { - logInfo("***dsv2-flows*** createWriter called " ) - - new JDBCDataWriter() + // TODO : Check if every task in executor call createWriter to gets its own writer object. + // JDBC connection should not be shared between Tasks. + // JDBCUtil.createConnectionFactory should take care of that?? + logInfo(s"***dsv2-flows*** createWriter called for partition $partitionId taskID $taskId") + val conn: Connection = JdbcUtils.createConnectionFactory(options)() + new JDBCDataWriter(options, conn, schema) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala index bb50a965edaa..48b1348a7f73 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala @@ -17,24 +17,97 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc +import java.sql.{Connection, DriverManager} + import org.apache.spark.internal.Logging -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, WriteBuilder} +import org.apache.spark.sql.SaveMode +import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.sources.{AlwaysTrue$, Filter} +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, SupportsOverwrite, WriteBuilder} import org.apache.spark.sql.types.StructType -class JDBCWriteBuilder extends WriteBuilder with Logging{ +class JDBCWriteBuilder(options: JdbcOptionsInWrite, + userSchema: Option[StructType]) + extends SupportsOverwrite with Logging{ + // TODO : Check, The default mode is assumed as Append. Refer physical plans to + // overwrite and append data i.e. OverwriteByExpressionExec and AppendDataExec + // respectively(Truncate and overwrite are called explicitly) + private var writeMode : SaveMode = SaveMode.Append + private var isTruncate : Boolean = false + private var fwPassedSchema : StructType = _ + private val conn : Connection = JdbcUtils.createConnectionFactory(options)() override def withQueryId(queryId: String): WriteBuilder = { logInfo("***dsv2-flows*** withQueryId called with queryId" + queryId) + // TODO : Check, Possible for his object to handles multiple queries on same table. this } override def withInputDataSchema(schema: StructType): WriteBuilder = { - logInfo("***dsv2-flows*** withInputDataSchema called with schema" + schema.printTreeString()) + logInfo("***dsv2-flows*** withInputDataSchema called with schema") + logInfo("***dsv2-flows*** schema is " + schema.printTreeString()) + fwPassedSchema = schema this } override def buildForBatch : BatchWrite = { - logInfo("***dsv2-flows*** BatchWrite called") - new JDBCBatchWrite() + logInfo("***dsv2-flows*** buildForBatch called") + writeMode match { + case SaveMode.Overwrite => + processOverwrite() + case SaveMode.Append => + processAppend() + } + new JDBCBatchWrite(options, fwPassedSchema) + } + + override def overwrite(filters: Array[Filter]): WriteBuilder = { + logInfo("***dsv2-flows*** overwrite called ") + writeMode = SaveMode.Overwrite + this + } + + override def truncate(): WriteBuilder = { + logInfo("***dsv2-flows*** overwrite called ") + writeMode = SaveMode.Overwrite + isTruncate = true + this + } + + def processOverwrite() : Boolean = { + /* Overwrite table logic + 1. Check if table exists. If not create it here. Should create be done?? + 2. If table exists and isTruncate, then just truncate existing table + 3. If table exists and !isTruncate, then recreate table with new schema + Post table creation, send requests to executors to insert data. + + check filters. + */ + logInfo("***dsv2-flows*** Overwrite table with new schema") + false + } + + def processAppend() : Unit = { + /* Append table logic + * 1. Check is table exists. Create if not. Step4. + * 2. If table exists and schema does not match, raise exception. + * 3. If table exists and schema match. Step4 + * 4. Send to executors for data insert + */ + logInfo("***dsv2-flows*** Append to table") + // log schemas received. + Utils.logSchema("userSchema", userSchema) + Utils.logSchema("fwPassedSchema", Option(fwPassedSchema)) + + JdbcUtils.tableExists(conn, options) match { + case true => + logInfo("***dsv2-flows*** Table exists" ) + Utils.strictSchemaCheck(fwPassedSchema) + logInfo("***dsv2-flows*** schema check done. Good to go." ) + case _ => + logInfo("***dsv2-flows*** Table does not exists." ) + // TODO : Check scemantics, Raise exception Or Create it. + Utils.createTable(fwPassedSchema) + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala new file mode 100644 index 000000000000..17909c3bd7b0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala @@ -0,0 +1,58 @@ +/* + * 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.jdbc + +import java.sql.{Connection, PreparedStatement} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils.{getCommonJDBCType, getJdbcType} +import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} +import org.apache.spark.sql.types.{BooleanType, ByteType, DataType, DoubleType, FloatType, IntegerType, LongType, ShortType, StringType, StructType} + +/* Misc utils + + */ +object Utils extends Logging{ + def logSchema(prefix: String, schema: Option[StructType]) : Unit = { + schema match { + case Some(i) => + val schemaString = i.printTreeString() + logInfo(s"***dsv2-flows*** $prefix schema exists" ) + logInfo(s"***dsv2-flows*** $prefix schemaInDB schema is $schemaString") + case None => + logInfo(s"***dsv2-flows*** $prefix schemaInDB schema is None" ) + } + } + + def createTable(structType: StructType): Unit = { + /* Create table per passed schema. Raise exception on any failure. + */ + } + + def strictSchemaCheck(schemaInSpark: StructType) : StructType = { + // TODO : Raise exception if fwPassedSchema is not same as schemaInDB. + schemaInSpark + } + + + + + +} From ad178565ab68dc95c1310e897016f6bc31878338 Mon Sep 17 00:00:00 2001 From: shivsood Date: Fri, 19 Jul 2019 15:45:39 -0700 Subject: [PATCH 03/24] first draft implementation of dataframe write(append) flow with datasourcev2. - df.write.format("jdbcv2").mode("append") appends to Table if table exists. create table not supported as of now. - validation with SQLServer 2017 only. - Good level of logging to help understand flows - E2E test cases added in MsSqlServerIntegrationSuite.scala --- .../jdbc/MsSqlServerIntegrationSuite.scala | 24 ++++++ .../datasources/jdbc/JdbcUtils.scala | 78 +++++++++++++++++ .../v2/jdbc/DBPartitionReaderFactory.scala | 1 - .../datasources/v2/jdbc/DBTable.scala | 52 ++++++------ .../datasources/v2/jdbc/JDBCBatchWrite.scala | 16 ++-- .../v2/jdbc/JDBCDataSourceV2.scala | 1 - .../datasources/v2/jdbc/JDBCDataWriter.scala | 22 ++--- .../v2/jdbc/JDBCDataWriterFactory.scala | 19 +++-- .../v2/jdbc/JDBCWriteBuilder.scala | 83 +++++++++++++++++-- .../execution/datasources/v2/jdbc/Utils.scala | 58 +++++++++++++ 10 files changed, 295 insertions(+), 59 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index efd7ca74c796..edc3cf0761e6 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -99,6 +99,30 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { |'the', 'lazy', |'dog') """.stripMargin).executeUpdate() + conn.prepareStatement( + """ + |CREATE TABLE strings_numbers ( + |i NVarChar(10), + |j INT, + |k NVarChar(20)) + """.stripMargin).executeUpdate() + conn.prepareStatement( + """ + |INSERT INTO strings_numbers VALUES ( + |'string',38, + |'big string') + """.stripMargin).executeUpdate() + } + + test("JDBCV2 write test") { + // Read 1 row using JDBC. Write(append) this row using jdbcv2. + val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() + df1.show(10) + assert(df1.count == 1) + df1.write.format("jdbcv2").mode("append").option("url",jdbcUrl).option("dbtable", "strings_numbers").save() + val df2 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() + df2.show(10) + assert(df2.count == 2) } test("Basic test") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 86a27b5afc25..63f62b31f89e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -870,4 +870,82 @@ object JdbcUtils extends Logging { statement.close() } } + + /** + * Inserts row into the database represented by the connection. + * + * @param conn - JDBCConnection to the given database. + * @param record - InternalRow to be inserted + * @throws IllegalArgumentException if the schema contains an unsupported type. + * Additions to JDBCUtils to support DSV2 JDBC adapter. + * TODO : Refactoring to remove makeIRSetter and use makeSetter with InternalRows + * TODO : Reactor to remove v2 imports/types from this pure DSV1 file. + * TODO : Support for Transactions, Isolation levels + * TODO : Just add to statement here. Commit should be outside of this function. + */ + def saveRow(conn: Connection, record: InternalRow, + options : JdbcOptionsInWrite, schema : StructType) : Unit = { + val dialect = JdbcDialects.get(options.url) + val rddSchema = JdbcUtils.getSchemaOption(conn, options) + val passedSchema = schema + + val insertStatement = JdbcUtils.getInsertStatement(options.table, + schema, rddSchema, true, dialect) + val stmt = conn.prepareStatement(insertStatement) + logInfo("***dsv2-flows*** insertStatement is $insertStatement") + val setters = schema.fields.map(f => makeIRSetter(conn, dialect, f.dataType)) + val nullTypes = schema.fields.map(f => getJdbcType(f.dataType, dialect).jdbcNullType) + val numFields = schema.fields.length + + var colNum = 0 + while (colNum < numFields) { + logInfo("***dsv2-flows*** stmt prep field " + colNum) + if (record.isNullAt(colNum)) { + stmt.setNull(colNum + 1, nullTypes(colNum)) + } else { + setters(colNum).apply(stmt, record, colNum) + } + colNum = colNum + 1 + } + stmt.execute() + } + + private type JDBCValueSetterIR = (PreparedStatement, InternalRow, Int) => Unit + + private def makeIRSetter(conn: Connection, dialect: JdbcDialect, + dataType: DataType): JDBCValueSetterIR = dataType match { + case IntegerType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setInt(pos + 1, row.getInt(pos)) + + case LongType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setLong(pos + 1, row.getLong(pos)) + + case DoubleType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setDouble(pos + 1, row.getDouble(pos)) + + case FloatType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setFloat(pos + 1, row.getFloat(pos)) + + case ShortType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setShort(pos + 1, row.getShort(pos)) + + case ByteType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setByte(pos + 1, row.getByte(pos)) + + case BooleanType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setBoolean(pos + 1, row.getBoolean(pos)) + + case StringType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setString(pos + 1, row.getString(pos)) + case _ => + throw new IllegalArgumentException(s"Not supported ${dataType.catalogString}") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala index 3c3d3a43e14a..03d86fa18229 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala @@ -28,5 +28,4 @@ class DBPartitionReaderFactory(schema : StructType) extends PartitionReaderFacto logInfo("***dsv2-flows*** createReader called") new DBPartitionReader(schema) } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala index 478ceb9c28cf..cea507454221 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -17,62 +17,60 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc +import java.sql.Connection + import scala.collection.JavaConverters._ import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.execution.datasources.v2.csv.CSVWriteBuilder +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions.JDBC_TABLE_NAME +import org.apache.spark.sql.execution.streaming.MetadataLogFileIndex import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.sources.v2.TableCapability.{BATCH_READ, BATCH_WRITE, TRUNCATE} +import org.apache.spark.sql.sources.v2.TableCapability.{BATCH_READ, BATCH_WRITE, OVERWRITE_BY_FILTER, TRUNCATE} import org.apache.spark.sql.sources.v2.reader.ScanBuilder import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types.{DataType, IntegerType, StringType, StructField, StructType} -import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql.util.{CaseInsensitiveStringMap, SchemaUtils} case class DBTable (sparkSession: SparkSession, - options: CaseInsensitiveStringMap, - userSpecifiedSchema: Option[StructType]) + options: CaseInsensitiveStringMap, + userSchema: Option[StructType]) extends Table with SupportsWrite with SupportsRead with Logging{ + private val userOptions = new JDBCOptions(options.asScala.toMap) + private val tableName = userOptions.parameters(JDBC_TABLE_NAME) + private val conn : Connection = JdbcUtils.createConnectionFactory(userOptions)() override def name: String = { - // TODO - Should come from user options - - logInfo("***dsv2-flows*** name called") - - "mysqltable" + logInfo("***dsv2-flows*** name called. Table name is " + tableName) + tableName } - def schema: StructType = { - // TODO - Remove hardcoded schema + override def schema: StructType = { + // TODO - check why a schema request? What if no table exists and + // no userSpecifiedSchema logInfo("***dsv2-flows*** schema called") - StructType(Seq( - StructField("name", StringType, true), - StructField("rollnum", StringType, true), - StructField("occupation", StringType, true))) + val schemaInDB = JdbcUtils.getSchemaOption(conn, userOptions) + Utils.logSchema("schema from DB", schemaInDB) + schemaInDB.getOrElse(StructType(Nil)) } override def capabilities: java.util.Set[TableCapability] = DBTable.CAPABILITIES - def supportsDataType(dataType: DataType): Boolean = true - override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { logInfo("***dsv2-flows*** newWriteBuilder called") - new JDBCWriteBuilder() + Utils.logSchema("Schema passed to DBTable", userSchema) + new JDBCWriteBuilder( + new JdbcOptionsInWrite(options.asScala.toMap), userSchema) } override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { logInfo("***dsv2-flows*** newScanBuilder called") new JDBCScanBuilder() } - } object DBTable { - private val CAPABILITIES = Set(BATCH_WRITE, BATCH_READ, TRUNCATE).asJava + private val CAPABILITIES = Set(BATCH_READ, BATCH_WRITE, TRUNCATE, OVERWRITE_BY_FILTER).asJava } - - - diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala index e4704b8bc737..952e05eb7530 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala @@ -18,25 +18,23 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.datasources.jdbc.JdbcOptionsInWrite import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.types.StructType -class JDBCBatchWrite extends BatchWrite with Logging{ +class JDBCBatchWrite(options: JdbcOptionsInWrite, fwPassedSchema: StructType) + extends BatchWrite with Logging{ def createBatchWriterFactory: DataWriterFactory = { logInfo("***dsv2-flows*** createBatchWriterFactory called" ) - new JDBCDataWriterFactory() + new JDBCDataWriterFactory(options, fwPassedSchema) } - - def commit(messages: Array[WriterCommitMessage]): Unit = { - - logInfo("***dsv2-flows*** commit called with message... " ) - + logInfo("***dsv2-flows*** commit called with $messages.length messages" + + s"with value as $messages.mkString(':')") } def abort(messages: Array[WriterCommitMessage]): Unit = { logInfo("***dsv2-flows*** abort called with message... " ) - } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala index 23ccc65ea082..aec0940b5bad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap class JDBCDataSourceV2 extends TableProvider with DataSourceRegister with Logging{ - override def shortName(): String = { logInfo("***dsv2-flows*** shortName - return connector name") "jdbcv2" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala index 91caf1a4058b..016921ca25c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala @@ -18,25 +18,24 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import java.io.IOException +import java.sql.{Connection, PreparedStatement} import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils._ +import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage} -import org.apache.spark.sql.types.{StringType, StructField, StructType} - -class JDBCDataWriter extends DataWriter[InternalRow] with Logging{ +import org.apache.spark.sql.types._ +class JDBCDataWriter(options: JdbcOptionsInWrite, + jdbcConn: Connection, schema: StructType) + extends DataWriter[InternalRow] with Logging{ @throws[IOException] def write(record: InternalRow): Unit = { logInfo("***dsv2-flows*** write " ) - - val schema = StructType(Seq( - StructField("name", StringType, true), - StructField("rollnum", StringType, true), - StructField("occupation", StringType, true))) - - val ret = record.toSeq(schema) - logInfo("***dsv2-flows*** write " + ret.mkString(":") ) + JdbcUtils.saveRow(jdbcConn, record, options, schema) } @throws[IOException] @@ -49,6 +48,7 @@ class JDBCDataWriter extends DataWriter[InternalRow] with Logging{ def abort(): Unit = { logInfo("***dsv2-flows*** abort called " ) } + } object JDBCWriterCommitMessage extends WriterCommitMessage { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala index f8f99716228f..db48840fba38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala @@ -17,16 +17,25 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc +import java.sql.Connection + import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory} +import org.apache.spark.sql.types.StructType -class JDBCDataWriterFactory extends DataWriterFactory with Logging{ - +/* Writer factory that's serialized and send to executors + */ +class JDBCDataWriterFactory(options: JdbcOptionsInWrite, schema: StructType) extends + DataWriterFactory with Logging{ def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = { - logInfo("***dsv2-flows*** createWriter called " ) - - new JDBCDataWriter() + // TODO : Check if every task in executor call createWriter to gets its own writer object. + // JDBC connection should not be shared between Tasks. + // JDBCUtil.createConnectionFactory should take care of that?? + logInfo(s"***dsv2-flows*** createWriter called for partition $partitionId taskID $taskId") + val conn: Connection = JdbcUtils.createConnectionFactory(options)() + new JDBCDataWriter(options, conn, schema) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala index bb50a965edaa..48b1348a7f73 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala @@ -17,24 +17,97 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc +import java.sql.{Connection, DriverManager} + import org.apache.spark.internal.Logging -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, WriteBuilder} +import org.apache.spark.sql.SaveMode +import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.sources.{AlwaysTrue$, Filter} +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, SupportsOverwrite, WriteBuilder} import org.apache.spark.sql.types.StructType -class JDBCWriteBuilder extends WriteBuilder with Logging{ +class JDBCWriteBuilder(options: JdbcOptionsInWrite, + userSchema: Option[StructType]) + extends SupportsOverwrite with Logging{ + // TODO : Check, The default mode is assumed as Append. Refer physical plans to + // overwrite and append data i.e. OverwriteByExpressionExec and AppendDataExec + // respectively(Truncate and overwrite are called explicitly) + private var writeMode : SaveMode = SaveMode.Append + private var isTruncate : Boolean = false + private var fwPassedSchema : StructType = _ + private val conn : Connection = JdbcUtils.createConnectionFactory(options)() override def withQueryId(queryId: String): WriteBuilder = { logInfo("***dsv2-flows*** withQueryId called with queryId" + queryId) + // TODO : Check, Possible for his object to handles multiple queries on same table. this } override def withInputDataSchema(schema: StructType): WriteBuilder = { - logInfo("***dsv2-flows*** withInputDataSchema called with schema" + schema.printTreeString()) + logInfo("***dsv2-flows*** withInputDataSchema called with schema") + logInfo("***dsv2-flows*** schema is " + schema.printTreeString()) + fwPassedSchema = schema this } override def buildForBatch : BatchWrite = { - logInfo("***dsv2-flows*** BatchWrite called") - new JDBCBatchWrite() + logInfo("***dsv2-flows*** buildForBatch called") + writeMode match { + case SaveMode.Overwrite => + processOverwrite() + case SaveMode.Append => + processAppend() + } + new JDBCBatchWrite(options, fwPassedSchema) + } + + override def overwrite(filters: Array[Filter]): WriteBuilder = { + logInfo("***dsv2-flows*** overwrite called ") + writeMode = SaveMode.Overwrite + this + } + + override def truncate(): WriteBuilder = { + logInfo("***dsv2-flows*** overwrite called ") + writeMode = SaveMode.Overwrite + isTruncate = true + this + } + + def processOverwrite() : Boolean = { + /* Overwrite table logic + 1. Check if table exists. If not create it here. Should create be done?? + 2. If table exists and isTruncate, then just truncate existing table + 3. If table exists and !isTruncate, then recreate table with new schema + Post table creation, send requests to executors to insert data. + + check filters. + */ + logInfo("***dsv2-flows*** Overwrite table with new schema") + false + } + + def processAppend() : Unit = { + /* Append table logic + * 1. Check is table exists. Create if not. Step4. + * 2. If table exists and schema does not match, raise exception. + * 3. If table exists and schema match. Step4 + * 4. Send to executors for data insert + */ + logInfo("***dsv2-flows*** Append to table") + // log schemas received. + Utils.logSchema("userSchema", userSchema) + Utils.logSchema("fwPassedSchema", Option(fwPassedSchema)) + + JdbcUtils.tableExists(conn, options) match { + case true => + logInfo("***dsv2-flows*** Table exists" ) + Utils.strictSchemaCheck(fwPassedSchema) + logInfo("***dsv2-flows*** schema check done. Good to go." ) + case _ => + logInfo("***dsv2-flows*** Table does not exists." ) + // TODO : Check scemantics, Raise exception Or Create it. + Utils.createTable(fwPassedSchema) + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala new file mode 100644 index 000000000000..17909c3bd7b0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala @@ -0,0 +1,58 @@ +/* + * 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.jdbc + +import java.sql.{Connection, PreparedStatement} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils.{getCommonJDBCType, getJdbcType} +import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} +import org.apache.spark.sql.types.{BooleanType, ByteType, DataType, DoubleType, FloatType, IntegerType, LongType, ShortType, StringType, StructType} + +/* Misc utils + + */ +object Utils extends Logging{ + def logSchema(prefix: String, schema: Option[StructType]) : Unit = { + schema match { + case Some(i) => + val schemaString = i.printTreeString() + logInfo(s"***dsv2-flows*** $prefix schema exists" ) + logInfo(s"***dsv2-flows*** $prefix schemaInDB schema is $schemaString") + case None => + logInfo(s"***dsv2-flows*** $prefix schemaInDB schema is None" ) + } + } + + def createTable(structType: StructType): Unit = { + /* Create table per passed schema. Raise exception on any failure. + */ + } + + def strictSchemaCheck(schemaInSpark: StructType) : StructType = { + // TODO : Raise exception if fwPassedSchema is not same as schemaInDB. + schemaInSpark + } + + + + + +} From 2ef896a51cbbf083a8aa912546facf54e7d8bc54 Mon Sep 17 00:00:00 2001 From: shivsood Date: Fri, 19 Jul 2019 16:43:40 -0700 Subject: [PATCH 04/24] Readme added with high level project work items and plan --- .../execution/datasources/v2/jdbc/Readme.md | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md new file mode 100644 index 000000000000..c51795a7933c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md @@ -0,0 +1,23 @@ +# Plan/Status of ongoing work on DataSource V2 JDBC connector + +## Plan +| Work Item | Who's on it | Status | +|-----------------------------------------------| ----------- | ------ | +| Batch write ( append, overwrite, truncate) | shivsood | WIP | +| Streaming write | TBD | | +| Read path implementation | TBD | | +| Streaming read | TBD | | +| ?? | TBD | | + +Status -> WIP ( Work in Progress), ReadyForReview, Done + +## Others +- Working branch is https://github.com/shivsood/spark-dsv2 +- Intrested in contribution? Add work item and your name against it and party on. + +## Major issues/mentions +- Lots of trivial logging. Draft implementation with API understanding as main goal + +Update date : 7/19 + + From 77448ab953299f8c05fbad87d55b877795311b80 Mon Sep 17 00:00:00 2001 From: shivsood Date: Mon, 15 Jul 2019 21:30:35 -0700 Subject: [PATCH 05/24] scaffolding : basic read/write with jdbc connector --- examples/src/main/python/sql/datasourcev2.py | 20 +++++ ...pache.spark.sql.sources.DataSourceRegister | 1 + .../v2/jdbc/DBPartitionReader.scala | 76 ++++++++++++++++++ .../v2/jdbc/DBPartitionReaderFactory.scala | 32 ++++++++ .../datasources/v2/jdbc/DBTable.scala | 78 +++++++++++++++++++ .../datasources/v2/jdbc/DBTableScan.scala | 56 +++++++++++++ .../datasources/v2/jdbc/JDBCBatchWrite.scala | 42 ++++++++++ .../v2/jdbc/JDBCDataSourceV2.scala | 43 ++++++++++ .../datasources/v2/jdbc/JDBCDataWriter.scala | 56 +++++++++++++ .../v2/jdbc/JDBCDataWriterFactory.scala | 32 ++++++++ .../datasources/v2/jdbc/JDBCScanBuilder.scala | 55 +++++++++++++ .../v2/jdbc/JDBCWriteBuilder.scala | 40 ++++++++++ 12 files changed, 531 insertions(+) create mode 100644 examples/src/main/python/sql/datasourcev2.py create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala diff --git a/examples/src/main/python/sql/datasourcev2.py b/examples/src/main/python/sql/datasourcev2.py new file mode 100644 index 000000000000..392d17ae00ff --- /dev/null +++ b/examples/src/main/python/sql/datasourcev2.py @@ -0,0 +1,20 @@ +from pyspark.sql import SparkSession +from pyspark.sql.functions import col + +spark = SparkSession\ + .builder\ + .appName("DV2Test")\ + .getOrCreate() + + +df = spark.read.format("csv").options(header='true').load("/home/shivsood/myspark_scripts/text.csv") +df.show(5) + +#Read +df_jdbc = spark.read.options(header='true').format("jdbcv2").load() +df_jdbc.show(3) + + +#Write +#df.filter( col("rollnum") == "38" ).write.format("jdbcv2").save() + diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index c0b8b270bab1..5ce0de75ec46 100644 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -9,3 +9,4 @@ org.apache.spark.sql.execution.streaming.ConsoleSinkProvider org.apache.spark.sql.execution.streaming.sources.RateStreamProvider org.apache.spark.sql.execution.streaming.sources.TextSocketSourceProvider org.apache.spark.sql.execution.datasources.binaryfile.BinaryFileFormat +org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCDataSourceV2 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala new file mode 100644 index 000000000000..9ce41711ca9b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala @@ -0,0 +1,76 @@ +/* + * 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.jdbc + +import java.io.IOException + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.unsafe.types.UTF8String + +class DBPartitionReader(schema : StructType) extends PartitionReader[InternalRow] with Logging { + + var dummyRows = 0 + + @throws[IOException] + def next(): Boolean = { + + logInfo("***dsv2-flows*** next() called") + + if(dummyRows <2) { + dummyRows = dummyRows + 1 + true + } else { + false + } + } + + def get: InternalRow = { + + logInfo("***dsv2-flows*** get() called for row " + dummyRows) + + // Value for row1 + var v_name = "shiv" + var v_rollnum = "38" + var v_occupation = "worker" + + if(dummyRows == 2) { + // Values for row2 + v_name = "someone" + v_rollnum = "39" + v_occupation = "dontknow" + } + + val values = schema.map(_.name).map { + case "name" => UTF8String.fromString(v_name) + case "rollnum" => UTF8String.fromString(v_rollnum) + case "occupation" => UTF8String.fromString(v_occupation) + case _ => UTF8String.fromString("anything") + } + + InternalRow.fromSeq(values) + } + + @throws[IOException] + override def close(): Unit = { + + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala new file mode 100644 index 000000000000..3c3d3a43e14a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala @@ -0,0 +1,32 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.types.StructType + +class DBPartitionReaderFactory(schema : StructType) extends PartitionReaderFactory with Logging{ + + def createReader(partition: InputPartition): PartitionReader[InternalRow] = { + logInfo("***dsv2-flows*** createReader called") + new DBPartitionReader(schema) + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala new file mode 100644 index 000000000000..478ceb9c28cf --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -0,0 +1,78 @@ +/* + * 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.jdbc + +import scala.collection.JavaConverters._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.v2.FileTable +import org.apache.spark.sql.execution.datasources.v2.csv.CSVWriteBuilder +import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.sources.v2.TableCapability.{BATCH_READ, BATCH_WRITE, TRUNCATE} +import org.apache.spark.sql.sources.v2.reader.ScanBuilder +import org.apache.spark.sql.sources.v2.writer.WriteBuilder +import org.apache.spark.sql.types.{DataType, IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class DBTable (sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + userSpecifiedSchema: Option[StructType]) + extends Table with SupportsWrite with SupportsRead with Logging{ + + + override def name: String = { + // TODO - Should come from user options + + logInfo("***dsv2-flows*** name called") + + "mysqltable" + } + + def schema: StructType = { + // TODO - Remove hardcoded schema + logInfo("***dsv2-flows*** schema called") + StructType(Seq( + StructField("name", StringType, true), + StructField("rollnum", StringType, true), + StructField("occupation", StringType, true))) + } + + override def capabilities: java.util.Set[TableCapability] = DBTable.CAPABILITIES + + def supportsDataType(dataType: DataType): Boolean = true + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + logInfo("***dsv2-flows*** newWriteBuilder called") + new JDBCWriteBuilder() + } + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + logInfo("***dsv2-flows*** newScanBuilder called") + new JDBCScanBuilder() + } + +} + +object DBTable { + private val CAPABILITIES = Set(BATCH_WRITE, BATCH_READ, TRUNCATE).asJava +} + + + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala new file mode 100644 index 000000000000..6b876dfc9429 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala @@ -0,0 +1,56 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReaderFactory, Scan} +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +class DBTableScan extends Scan with Batch with Logging { + + val table_schema = StructType(Seq( + StructField("name", StringType, true), + StructField("rollnum", StringType, true), + StructField("occupation", StringType, true))) + + def readSchema: StructType = { + logInfo("***dsv2-flows*** readSchema called") + table_schema + + } + + override def toBatch() : Batch = { + this + } + + def planInputPartitions: Array[InputPartition] = { + Array(PartitionScheme) + } + + def createReaderFactory: PartitionReaderFactory = { + + logInfo("***dsv2-flows*** createReaderFactory called") + new DBPartitionReaderFactory(table_schema) + + } + +} + +object PartitionScheme extends InputPartition { + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala new file mode 100644 index 000000000000..e4704b8bc737 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala @@ -0,0 +1,42 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, WriterCommitMessage} + +class JDBCBatchWrite extends BatchWrite with Logging{ + + def createBatchWriterFactory: DataWriterFactory = { + logInfo("***dsv2-flows*** createBatchWriterFactory called" ) + new JDBCDataWriterFactory() + } + + + def commit(messages: Array[WriterCommitMessage]): Unit = { + + logInfo("***dsv2-flows*** commit called with message... " ) + + } + + def abort(messages: Array[WriterCommitMessage]): Unit = { + logInfo("***dsv2-flows*** abort called with message... " ) + + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala new file mode 100644 index 000000000000..23ccc65ea082 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala @@ -0,0 +1,43 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.sources.v2.{Table, TableProvider} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class JDBCDataSourceV2 extends TableProvider with DataSourceRegister with Logging{ + + override def shortName(): String = { + logInfo("***dsv2-flows*** shortName - return connector name") + "jdbcv2" + } + + override def getTable(options: CaseInsensitiveStringMap): Table = { + logInfo("***dsv2-flows*** getTable called") + DBTable(SparkSession.active, options, None) + } + + override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { + logInfo("***dsv2-flows*** getTable called with schema") + DBTable(SparkSession.active, options, Some(schema)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala new file mode 100644 index 000000000000..91caf1a4058b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala @@ -0,0 +1,56 @@ +/* + * 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.jdbc + +import java.io.IOException + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage} +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +class JDBCDataWriter extends DataWriter[InternalRow] with Logging{ + + @throws[IOException] + def write(record: InternalRow): Unit = { + logInfo("***dsv2-flows*** write " ) + + val schema = StructType(Seq( + StructField("name", StringType, true), + StructField("rollnum", StringType, true), + StructField("occupation", StringType, true))) + + val ret = record.toSeq(schema) + logInfo("***dsv2-flows*** write " + ret.mkString(":") ) + } + + @throws[IOException] + def commit: WriterCommitMessage = { + logInfo("***dsv2-flows*** commit called " ) + JDBCWriterCommitMessage + } + + @throws[IOException] + def abort(): Unit = { + logInfo("***dsv2-flows*** abort called " ) + } +} + +object JDBCWriterCommitMessage extends WriterCommitMessage { + val commitMessage: String = "committed" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala new file mode 100644 index 000000000000..f8f99716228f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala @@ -0,0 +1,32 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory} + +class JDBCDataWriterFactory extends DataWriterFactory with Logging{ + + def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = { + logInfo("***dsv2-flows*** createWriter called " ) + + new JDBCDataWriter() + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala new file mode 100644 index 000000000000..ac927b70dde7 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala @@ -0,0 +1,55 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.v2.SupportsRead +import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.types.StructType + +class JDBCScanBuilder extends ScanBuilder with + SupportsPushDownFilters with SupportsPushDownRequiredColumns + with Logging { + + var specifiedFilters: Array[Filter] = Array.empty + + + def build: Scan = { + logInfo("***dsv2-flows*** Scan called") + new DBTableScan() + + } + + def pushFilters(filters: Array[Filter]): Array[Filter] = { + logInfo("***dsv2-flows*** PushDown filters called") + specifiedFilters = filters + filters + } + + def pruneColumns(requiredSchema: StructType): Unit = { + logInfo("***dsv2-flows*** pruneColumns called") + + } + + def pushedFilters: Array[Filter] = { + logInfo("***dsv2-flows*** pushedFilters called") + specifiedFilters + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala new file mode 100644 index 000000000000..bb50a965edaa --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala @@ -0,0 +1,40 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, WriteBuilder} +import org.apache.spark.sql.types.StructType + +class JDBCWriteBuilder extends WriteBuilder with Logging{ + + override def withQueryId(queryId: String): WriteBuilder = { + logInfo("***dsv2-flows*** withQueryId called with queryId" + queryId) + this + } + + override def withInputDataSchema(schema: StructType): WriteBuilder = { + logInfo("***dsv2-flows*** withInputDataSchema called with schema" + schema.printTreeString()) + this + } + + override def buildForBatch : BatchWrite = { + logInfo("***dsv2-flows*** BatchWrite called") + new JDBCBatchWrite() + } +} From 675083f4d60e99c397f7177d224d1413f5e97913 Mon Sep 17 00:00:00 2001 From: shivsood Date: Fri, 19 Jul 2019 15:45:39 -0700 Subject: [PATCH 06/24] first draft implemnentation of write(append) flow with datasourcev2. E2E test case added in MssQLServerIntegrationSuite --- .../jdbc/MsSqlServerIntegrationSuite.scala | 24 ++++++ .../datasources/jdbc/JdbcUtils.scala | 78 +++++++++++++++++ .../v2/jdbc/DBPartitionReaderFactory.scala | 1 - .../datasources/v2/jdbc/DBTable.scala | 52 ++++++------ .../datasources/v2/jdbc/JDBCBatchWrite.scala | 16 ++-- .../v2/jdbc/JDBCDataSourceV2.scala | 1 - .../datasources/v2/jdbc/JDBCDataWriter.scala | 22 ++--- .../v2/jdbc/JDBCDataWriterFactory.scala | 19 +++-- .../v2/jdbc/JDBCWriteBuilder.scala | 83 +++++++++++++++++-- .../execution/datasources/v2/jdbc/Utils.scala | 58 +++++++++++++ 10 files changed, 295 insertions(+), 59 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index efd7ca74c796..edc3cf0761e6 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -99,6 +99,30 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { |'the', 'lazy', |'dog') """.stripMargin).executeUpdate() + conn.prepareStatement( + """ + |CREATE TABLE strings_numbers ( + |i NVarChar(10), + |j INT, + |k NVarChar(20)) + """.stripMargin).executeUpdate() + conn.prepareStatement( + """ + |INSERT INTO strings_numbers VALUES ( + |'string',38, + |'big string') + """.stripMargin).executeUpdate() + } + + test("JDBCV2 write test") { + // Read 1 row using JDBC. Write(append) this row using jdbcv2. + val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() + df1.show(10) + assert(df1.count == 1) + df1.write.format("jdbcv2").mode("append").option("url",jdbcUrl).option("dbtable", "strings_numbers").save() + val df2 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() + df2.show(10) + assert(df2.count == 2) } test("Basic test") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 86a27b5afc25..63f62b31f89e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -870,4 +870,82 @@ object JdbcUtils extends Logging { statement.close() } } + + /** + * Inserts row into the database represented by the connection. + * + * @param conn - JDBCConnection to the given database. + * @param record - InternalRow to be inserted + * @throws IllegalArgumentException if the schema contains an unsupported type. + * Additions to JDBCUtils to support DSV2 JDBC adapter. + * TODO : Refactoring to remove makeIRSetter and use makeSetter with InternalRows + * TODO : Reactor to remove v2 imports/types from this pure DSV1 file. + * TODO : Support for Transactions, Isolation levels + * TODO : Just add to statement here. Commit should be outside of this function. + */ + def saveRow(conn: Connection, record: InternalRow, + options : JdbcOptionsInWrite, schema : StructType) : Unit = { + val dialect = JdbcDialects.get(options.url) + val rddSchema = JdbcUtils.getSchemaOption(conn, options) + val passedSchema = schema + + val insertStatement = JdbcUtils.getInsertStatement(options.table, + schema, rddSchema, true, dialect) + val stmt = conn.prepareStatement(insertStatement) + logInfo("***dsv2-flows*** insertStatement is $insertStatement") + val setters = schema.fields.map(f => makeIRSetter(conn, dialect, f.dataType)) + val nullTypes = schema.fields.map(f => getJdbcType(f.dataType, dialect).jdbcNullType) + val numFields = schema.fields.length + + var colNum = 0 + while (colNum < numFields) { + logInfo("***dsv2-flows*** stmt prep field " + colNum) + if (record.isNullAt(colNum)) { + stmt.setNull(colNum + 1, nullTypes(colNum)) + } else { + setters(colNum).apply(stmt, record, colNum) + } + colNum = colNum + 1 + } + stmt.execute() + } + + private type JDBCValueSetterIR = (PreparedStatement, InternalRow, Int) => Unit + + private def makeIRSetter(conn: Connection, dialect: JdbcDialect, + dataType: DataType): JDBCValueSetterIR = dataType match { + case IntegerType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setInt(pos + 1, row.getInt(pos)) + + case LongType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setLong(pos + 1, row.getLong(pos)) + + case DoubleType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setDouble(pos + 1, row.getDouble(pos)) + + case FloatType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setFloat(pos + 1, row.getFloat(pos)) + + case ShortType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setShort(pos + 1, row.getShort(pos)) + + case ByteType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setByte(pos + 1, row.getByte(pos)) + + case BooleanType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setBoolean(pos + 1, row.getBoolean(pos)) + + case StringType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setString(pos + 1, row.getString(pos)) + case _ => + throw new IllegalArgumentException(s"Not supported ${dataType.catalogString}") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala index 3c3d3a43e14a..03d86fa18229 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala @@ -28,5 +28,4 @@ class DBPartitionReaderFactory(schema : StructType) extends PartitionReaderFacto logInfo("***dsv2-flows*** createReader called") new DBPartitionReader(schema) } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala index 478ceb9c28cf..cea507454221 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -17,62 +17,60 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc +import java.sql.Connection + import scala.collection.JavaConverters._ import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.execution.datasources.v2.csv.CSVWriteBuilder +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions.JDBC_TABLE_NAME +import org.apache.spark.sql.execution.streaming.MetadataLogFileIndex import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.sources.v2.TableCapability.{BATCH_READ, BATCH_WRITE, TRUNCATE} +import org.apache.spark.sql.sources.v2.TableCapability.{BATCH_READ, BATCH_WRITE, OVERWRITE_BY_FILTER, TRUNCATE} import org.apache.spark.sql.sources.v2.reader.ScanBuilder import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types.{DataType, IntegerType, StringType, StructField, StructType} -import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql.util.{CaseInsensitiveStringMap, SchemaUtils} case class DBTable (sparkSession: SparkSession, - options: CaseInsensitiveStringMap, - userSpecifiedSchema: Option[StructType]) + options: CaseInsensitiveStringMap, + userSchema: Option[StructType]) extends Table with SupportsWrite with SupportsRead with Logging{ + private val userOptions = new JDBCOptions(options.asScala.toMap) + private val tableName = userOptions.parameters(JDBC_TABLE_NAME) + private val conn : Connection = JdbcUtils.createConnectionFactory(userOptions)() override def name: String = { - // TODO - Should come from user options - - logInfo("***dsv2-flows*** name called") - - "mysqltable" + logInfo("***dsv2-flows*** name called. Table name is " + tableName) + tableName } - def schema: StructType = { - // TODO - Remove hardcoded schema + override def schema: StructType = { + // TODO - check why a schema request? What if no table exists and + // no userSpecifiedSchema logInfo("***dsv2-flows*** schema called") - StructType(Seq( - StructField("name", StringType, true), - StructField("rollnum", StringType, true), - StructField("occupation", StringType, true))) + val schemaInDB = JdbcUtils.getSchemaOption(conn, userOptions) + Utils.logSchema("schema from DB", schemaInDB) + schemaInDB.getOrElse(StructType(Nil)) } override def capabilities: java.util.Set[TableCapability] = DBTable.CAPABILITIES - def supportsDataType(dataType: DataType): Boolean = true - override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { logInfo("***dsv2-flows*** newWriteBuilder called") - new JDBCWriteBuilder() + Utils.logSchema("Schema passed to DBTable", userSchema) + new JDBCWriteBuilder( + new JdbcOptionsInWrite(options.asScala.toMap), userSchema) } override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { logInfo("***dsv2-flows*** newScanBuilder called") new JDBCScanBuilder() } - } object DBTable { - private val CAPABILITIES = Set(BATCH_WRITE, BATCH_READ, TRUNCATE).asJava + private val CAPABILITIES = Set(BATCH_READ, BATCH_WRITE, TRUNCATE, OVERWRITE_BY_FILTER).asJava } - - - diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala index e4704b8bc737..952e05eb7530 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala @@ -18,25 +18,23 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.datasources.jdbc.JdbcOptionsInWrite import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.types.StructType -class JDBCBatchWrite extends BatchWrite with Logging{ +class JDBCBatchWrite(options: JdbcOptionsInWrite, fwPassedSchema: StructType) + extends BatchWrite with Logging{ def createBatchWriterFactory: DataWriterFactory = { logInfo("***dsv2-flows*** createBatchWriterFactory called" ) - new JDBCDataWriterFactory() + new JDBCDataWriterFactory(options, fwPassedSchema) } - - def commit(messages: Array[WriterCommitMessage]): Unit = { - - logInfo("***dsv2-flows*** commit called with message... " ) - + logInfo("***dsv2-flows*** commit called with $messages.length messages" + + s"with value as $messages.mkString(':')") } def abort(messages: Array[WriterCommitMessage]): Unit = { logInfo("***dsv2-flows*** abort called with message... " ) - } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala index 23ccc65ea082..aec0940b5bad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap class JDBCDataSourceV2 extends TableProvider with DataSourceRegister with Logging{ - override def shortName(): String = { logInfo("***dsv2-flows*** shortName - return connector name") "jdbcv2" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala index 91caf1a4058b..016921ca25c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala @@ -18,25 +18,24 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import java.io.IOException +import java.sql.{Connection, PreparedStatement} import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils._ +import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage} -import org.apache.spark.sql.types.{StringType, StructField, StructType} - -class JDBCDataWriter extends DataWriter[InternalRow] with Logging{ +import org.apache.spark.sql.types._ +class JDBCDataWriter(options: JdbcOptionsInWrite, + jdbcConn: Connection, schema: StructType) + extends DataWriter[InternalRow] with Logging{ @throws[IOException] def write(record: InternalRow): Unit = { logInfo("***dsv2-flows*** write " ) - - val schema = StructType(Seq( - StructField("name", StringType, true), - StructField("rollnum", StringType, true), - StructField("occupation", StringType, true))) - - val ret = record.toSeq(schema) - logInfo("***dsv2-flows*** write " + ret.mkString(":") ) + JdbcUtils.saveRow(jdbcConn, record, options, schema) } @throws[IOException] @@ -49,6 +48,7 @@ class JDBCDataWriter extends DataWriter[InternalRow] with Logging{ def abort(): Unit = { logInfo("***dsv2-flows*** abort called " ) } + } object JDBCWriterCommitMessage extends WriterCommitMessage { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala index f8f99716228f..db48840fba38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala @@ -17,16 +17,25 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc +import java.sql.Connection + import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory} +import org.apache.spark.sql.types.StructType -class JDBCDataWriterFactory extends DataWriterFactory with Logging{ - +/* Writer factory that's serialized and send to executors + */ +class JDBCDataWriterFactory(options: JdbcOptionsInWrite, schema: StructType) extends + DataWriterFactory with Logging{ def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = { - logInfo("***dsv2-flows*** createWriter called " ) - - new JDBCDataWriter() + // TODO : Check if every task in executor call createWriter to gets its own writer object. + // JDBC connection should not be shared between Tasks. + // JDBCUtil.createConnectionFactory should take care of that?? + logInfo(s"***dsv2-flows*** createWriter called for partition $partitionId taskID $taskId") + val conn: Connection = JdbcUtils.createConnectionFactory(options)() + new JDBCDataWriter(options, conn, schema) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala index bb50a965edaa..48b1348a7f73 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala @@ -17,24 +17,97 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc +import java.sql.{Connection, DriverManager} + import org.apache.spark.internal.Logging -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, WriteBuilder} +import org.apache.spark.sql.SaveMode +import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.sources.{AlwaysTrue$, Filter} +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, SupportsOverwrite, WriteBuilder} import org.apache.spark.sql.types.StructType -class JDBCWriteBuilder extends WriteBuilder with Logging{ +class JDBCWriteBuilder(options: JdbcOptionsInWrite, + userSchema: Option[StructType]) + extends SupportsOverwrite with Logging{ + // TODO : Check, The default mode is assumed as Append. Refer physical plans to + // overwrite and append data i.e. OverwriteByExpressionExec and AppendDataExec + // respectively(Truncate and overwrite are called explicitly) + private var writeMode : SaveMode = SaveMode.Append + private var isTruncate : Boolean = false + private var fwPassedSchema : StructType = _ + private val conn : Connection = JdbcUtils.createConnectionFactory(options)() override def withQueryId(queryId: String): WriteBuilder = { logInfo("***dsv2-flows*** withQueryId called with queryId" + queryId) + // TODO : Check, Possible for his object to handles multiple queries on same table. this } override def withInputDataSchema(schema: StructType): WriteBuilder = { - logInfo("***dsv2-flows*** withInputDataSchema called with schema" + schema.printTreeString()) + logInfo("***dsv2-flows*** withInputDataSchema called with schema") + logInfo("***dsv2-flows*** schema is " + schema.printTreeString()) + fwPassedSchema = schema this } override def buildForBatch : BatchWrite = { - logInfo("***dsv2-flows*** BatchWrite called") - new JDBCBatchWrite() + logInfo("***dsv2-flows*** buildForBatch called") + writeMode match { + case SaveMode.Overwrite => + processOverwrite() + case SaveMode.Append => + processAppend() + } + new JDBCBatchWrite(options, fwPassedSchema) + } + + override def overwrite(filters: Array[Filter]): WriteBuilder = { + logInfo("***dsv2-flows*** overwrite called ") + writeMode = SaveMode.Overwrite + this + } + + override def truncate(): WriteBuilder = { + logInfo("***dsv2-flows*** overwrite called ") + writeMode = SaveMode.Overwrite + isTruncate = true + this + } + + def processOverwrite() : Boolean = { + /* Overwrite table logic + 1. Check if table exists. If not create it here. Should create be done?? + 2. If table exists and isTruncate, then just truncate existing table + 3. If table exists and !isTruncate, then recreate table with new schema + Post table creation, send requests to executors to insert data. + + check filters. + */ + logInfo("***dsv2-flows*** Overwrite table with new schema") + false + } + + def processAppend() : Unit = { + /* Append table logic + * 1. Check is table exists. Create if not. Step4. + * 2. If table exists and schema does not match, raise exception. + * 3. If table exists and schema match. Step4 + * 4. Send to executors for data insert + */ + logInfo("***dsv2-flows*** Append to table") + // log schemas received. + Utils.logSchema("userSchema", userSchema) + Utils.logSchema("fwPassedSchema", Option(fwPassedSchema)) + + JdbcUtils.tableExists(conn, options) match { + case true => + logInfo("***dsv2-flows*** Table exists" ) + Utils.strictSchemaCheck(fwPassedSchema) + logInfo("***dsv2-flows*** schema check done. Good to go." ) + case _ => + logInfo("***dsv2-flows*** Table does not exists." ) + // TODO : Check scemantics, Raise exception Or Create it. + Utils.createTable(fwPassedSchema) + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala new file mode 100644 index 000000000000..17909c3bd7b0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala @@ -0,0 +1,58 @@ +/* + * 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.jdbc + +import java.sql.{Connection, PreparedStatement} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils.{getCommonJDBCType, getJdbcType} +import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} +import org.apache.spark.sql.types.{BooleanType, ByteType, DataType, DoubleType, FloatType, IntegerType, LongType, ShortType, StringType, StructType} + +/* Misc utils + + */ +object Utils extends Logging{ + def logSchema(prefix: String, schema: Option[StructType]) : Unit = { + schema match { + case Some(i) => + val schemaString = i.printTreeString() + logInfo(s"***dsv2-flows*** $prefix schema exists" ) + logInfo(s"***dsv2-flows*** $prefix schemaInDB schema is $schemaString") + case None => + logInfo(s"***dsv2-flows*** $prefix schemaInDB schema is None" ) + } + } + + def createTable(structType: StructType): Unit = { + /* Create table per passed schema. Raise exception on any failure. + */ + } + + def strictSchemaCheck(schemaInSpark: StructType) : StructType = { + // TODO : Raise exception if fwPassedSchema is not same as schemaInDB. + schemaInSpark + } + + + + + +} From 90e1ad97318f64c2fa0b7762e65eaefd04732499 Mon Sep 17 00:00:00 2001 From: shivsood Date: Fri, 19 Jul 2019 16:43:40 -0700 Subject: [PATCH 07/24] Readme added with high level project work items and plan --- .../execution/datasources/v2/jdbc/Readme.md | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md new file mode 100644 index 000000000000..c51795a7933c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md @@ -0,0 +1,23 @@ +# Plan/Status of ongoing work on DataSource V2 JDBC connector + +## Plan +| Work Item | Who's on it | Status | +|-----------------------------------------------| ----------- | ------ | +| Batch write ( append, overwrite, truncate) | shivsood | WIP | +| Streaming write | TBD | | +| Read path implementation | TBD | | +| Streaming read | TBD | | +| ?? | TBD | | + +Status -> WIP ( Work in Progress), ReadyForReview, Done + +## Others +- Working branch is https://github.com/shivsood/spark-dsv2 +- Intrested in contribution? Add work item and your name against it and party on. + +## Major issues/mentions +- Lots of trivial logging. Draft implementation with API understanding as main goal + +Update date : 7/19 + + From 5ddc5e9a681035aaf6098301f934c1cce87ff466 Mon Sep 17 00:00:00 2001 From: shivsood Date: Sat, 20 Jul 2019 10:23:54 -0700 Subject: [PATCH 08/24] hygiene fixes and comments clarifing that read implementation is dummy for now --- .../datasources/v2/jdbc/DBPartitionReader.scala | 13 ++++++------- .../v2/jdbc/DBPartitionReaderFactory.scala | 1 - .../execution/datasources/v2/jdbc/DBTableScan.scala | 9 +++++---- .../datasources/v2/jdbc/JDBCScanBuilder.scala | 1 - .../sql/execution/datasources/v2/jdbc/Utils.scala | 5 ----- 5 files changed, 11 insertions(+), 18 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala index 9ce41711ca9b..0fcc9f490815 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala @@ -26,14 +26,16 @@ import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.unsafe.types.UTF8String class DBPartitionReader(schema : StructType) extends PartitionReader[InternalRow] with Logging { - var dummyRows = 0 + /* + * Note : Read implementation is dummy as of now. + * It returns a hard coded schema and rows. + */ @throws[IOException] def next(): Boolean = { logInfo("***dsv2-flows*** next() called") - if(dummyRows <2) { dummyRows = dummyRows + 1 true @@ -47,7 +49,7 @@ class DBPartitionReader(schema : StructType) extends PartitionReader[InternalRow logInfo("***dsv2-flows*** get() called for row " + dummyRows) // Value for row1 - var v_name = "shiv" + var v_name = "somename" var v_rollnum = "38" var v_occupation = "worker" @@ -55,7 +57,7 @@ class DBPartitionReader(schema : StructType) extends PartitionReader[InternalRow // Values for row2 v_name = "someone" v_rollnum = "39" - v_occupation = "dontknow" + v_occupation = "manager" } val values = schema.map(_.name).map { @@ -64,13 +66,10 @@ class DBPartitionReader(schema : StructType) extends PartitionReader[InternalRow case "occupation" => UTF8String.fromString(v_occupation) case _ => UTF8String.fromString("anything") } - InternalRow.fromSeq(values) } @throws[IOException] override def close(): Unit = { - } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala index 03d86fa18229..ccf1e681544e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, import org.apache.spark.sql.types.StructType class DBPartitionReaderFactory(schema : StructType) extends PartitionReaderFactory with Logging{ - def createReader(partition: InputPartition): PartitionReader[InternalRow] = { logInfo("***dsv2-flows*** createReader called") new DBPartitionReader(schema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala index 6b876dfc9429..cc757867e567 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala @@ -23,6 +23,11 @@ import org.apache.spark.sql.types.{StringType, StructField, StructType} class DBTableScan extends Scan with Batch with Logging { + /* + * Note : Read implementation is dummy as of now. + * It returns a hard coded schema and rows. + */ + val table_schema = StructType(Seq( StructField("name", StringType, true), StructField("rollnum", StringType, true), @@ -43,14 +48,10 @@ class DBTableScan extends Scan with Batch with Logging { } def createReaderFactory: PartitionReaderFactory = { - logInfo("***dsv2-flows*** createReaderFactory called") new DBPartitionReaderFactory(table_schema) - } - } object PartitionScheme extends InputPartition { - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala index ac927b70dde7..6526840ebb22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala @@ -29,7 +29,6 @@ class JDBCScanBuilder extends ScanBuilder with var specifiedFilters: Array[Filter] = Array.empty - def build: Scan = { logInfo("***dsv2-flows*** Scan called") new DBTableScan() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala index 17909c3bd7b0..c43fb46f1e18 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala @@ -50,9 +50,4 @@ object Utils extends Logging{ // TODO : Raise exception if fwPassedSchema is not same as schemaInDB. schemaInSpark } - - - - - } From 57baf76f08a2427963cf9326fe2429d01808ace4 Mon Sep 17 00:00:00 2001 From: shivsood Date: Mon, 22 Jul 2019 14:41:56 -0700 Subject: [PATCH 09/24] cleaned up write(append) implementation. Append will not create a table and only append if a table exist. If table does not exist, a dbtable::schema request would return a null schema and fw will raise an exception. --- .../jdbc/MsSqlServerIntegrationSuite.scala | 26 ++++++++++++++++++- .../datasources/v2/jdbc/DBTable.scala | 6 +++-- .../v2/jdbc/JDBCWriteBuilder.scala | 18 ++----------- .../execution/datasources/v2/jdbc/Utils.scala | 12 +++++++-- 4 files changed, 41 insertions(+), 21 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index edc3cf0761e6..2f6f18171d3f 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -18,10 +18,12 @@ package org.apache.spark.sql.jdbc import java.math.BigDecimal -import java.sql.{Connection, Date, Timestamp} +import java.sql.{Connection, Date, Struct, Timestamp} import java.util.Properties import org.apache.spark.tags.DockerTest +import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.types._ @DockerTest class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { @@ -114,6 +116,20 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { """.stripMargin).executeUpdate() } + def create_test_df() : DataFrame = { + val schema:StructType = StructType( + Seq(StructField ("i", IntegerType, true), + StructField ("j", IntegerType, true), + StructField ("k", IntegerType, true)) + ) + val data:Seq[Row] = Seq( + Row(1,1,2), + Row(1,2,3) + ) + + spark.createDataFrame(spark.sparkContext.parallelize(data),schema) + } + test("JDBCV2 write test") { // Read 1 row using JDBC. Write(append) this row using jdbcv2. val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() @@ -123,6 +139,14 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { val df2 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() df2.show(10) assert(df2.count == 2) + + // Create a df with diffirent schema and append this to existing table. No convinced why this + // is passing. writing a dataframe with diffirent schema should fail. + val df_new = create_test_df() + df_new.write.format("jdbcv2").mode("append").option("url",jdbcUrl).option("dbtable", "strings_numbers").save() + val df2_new = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() + df2_new.show(10) + assert(df2_new.count == 4) } test("Basic test") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala index cea507454221..cb5077499ad6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -48,8 +48,10 @@ case class DBTable (sparkSession: SparkSession, } override def schema: StructType = { - // TODO - check why a schema request? What if no table exists and - // no userSpecifiedSchema + /* TODO - check why a schema request? Will this be called for every 'append' + * request to get schema from DBTable. If so, schema check in append should + * not be required. + */ logInfo("***dsv2-flows*** schema called") val schemaInDB = JdbcUtils.getSchemaOption(conn, userOptions) Utils.logSchema("schema from DB", schemaInDB) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala index 48b1348a7f73..52174a4e96fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala @@ -88,26 +88,12 @@ class JDBCWriteBuilder(options: JdbcOptionsInWrite, } def processAppend() : Unit = { - /* Append table logic - * 1. Check is table exists. Create if not. Step4. - * 2. If table exists and schema does not match, raise exception. - * 3. If table exists and schema match. Step4 - * 4. Send to executors for data insert + /* Append table logic : If we have reached this far, table exist and schema check is done. + * So processappend does nothing here. just sends request to executors for data insert */ logInfo("***dsv2-flows*** Append to table") // log schemas received. Utils.logSchema("userSchema", userSchema) Utils.logSchema("fwPassedSchema", Option(fwPassedSchema)) - - JdbcUtils.tableExists(conn, options) match { - case true => - logInfo("***dsv2-flows*** Table exists" ) - Utils.strictSchemaCheck(fwPassedSchema) - logInfo("***dsv2-flows*** schema check done. Good to go." ) - case _ => - logInfo("***dsv2-flows*** Table does not exists." ) - // TODO : Check scemantics, Raise exception Or Create it. - Utils.createTable(fwPassedSchema) - } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala index c43fb46f1e18..2c64093a2045 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import java.sql.{Connection, PreparedStatement} import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils.{getCommonJDBCType, getJdbcType} @@ -46,8 +47,15 @@ object Utils extends Logging{ */ } - def strictSchemaCheck(schemaInSpark: StructType) : StructType = { + def strictSchemaCheck(schemaInSpark: StructType, dbTableSchema: StructType) : Boolean = { // TODO : Raise exception if fwPassedSchema is not same as schemaInDB. - schemaInSpark + if (schemaInSpark == dbTableSchema) { + logInfo(s"***dsv2-flows*** strictSchemaCheck passed" ) + true + } else { + logInfo(s"***dsv2-flows*** schema check failed" ) + throw new AnalysisException( + s"Schema does not match with that with the database table") + } } } From 7246534bc78a0172963c71386939d29bbf9b82c6 Mon Sep 17 00:00:00 2001 From: shivsood Date: Mon, 22 Jul 2019 16:23:52 -0700 Subject: [PATCH 10/24] comments/questions on overwrite scematics and minor log fixes --- .../sql/jdbc/MsSqlServerIntegrationSuite.scala | 11 ++++++++++- .../execution/datasources/v2/jdbc/DBTable.scala | 9 ++++++--- .../datasources/v2/jdbc/JDBCWriteBuilder.scala | 5 +++-- .../sql/execution/datasources/v2/jdbc/Readme.md | 15 +++++++++++++-- 4 files changed, 32 insertions(+), 8 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index 2f6f18171d3f..3d8e68b6bb37 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -130,7 +130,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { spark.createDataFrame(spark.sparkContext.parallelize(data),schema) } - test("JDBCV2 write test") { + test("JDBCV2 write append test") { // Read 1 row using JDBC. Write(append) this row using jdbcv2. val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() df1.show(10) @@ -149,6 +149,15 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { assert(df2_new.count == 4) } + test("JDBCV2 write overwrite test") { + // Overwrite a existing table with a new schema and values. + val df1 = create_test_df() + // Overwrite test. Overwrite mode create a new table if it does not exist + df1.write.format("jdbcv2").mode("overwrite").option("url",jdbcUrl).option("dbtable","strings_numbers").save() + val df2 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable","strings_numbers").load() + df2.show() + } + test("Basic test") { val df = spark.read.jdbc(jdbcUrl, "tbl", new Properties) val rows = df.collect() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala index cb5077499ad6..296b120e030d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -48,9 +48,12 @@ case class DBTable (sparkSession: SparkSession, } override def schema: StructType = { - /* TODO - check why a schema request? Will this be called for every 'append' - * request to get schema from DBTable. If so, schema check in append should - * not be required. + /* TODO - check why a schema request? + * 1. Will this be called for every 'append' request to get schema from DBTable. + * If so, schema check in append should not be required. + * 2. This is called in overwrite as well. What if the table does not exist. A empty + * schema would be returned resulting in an exception. Should Overwrite semantics + * not create a table. */ logInfo("***dsv2-flows*** schema called") val schemaInDB = JdbcUtils.getSchemaOption(conn, userOptions) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala index 52174a4e96fb..2dd141df70d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala @@ -64,11 +64,12 @@ class JDBCWriteBuilder(options: JdbcOptionsInWrite, override def overwrite(filters: Array[Filter]): WriteBuilder = { logInfo("***dsv2-flows*** overwrite called ") writeMode = SaveMode.Overwrite + for(filter <- filters) logInfo(s"***dsv2-flows*** overwrite filter is $filter") this } override def truncate(): WriteBuilder = { - logInfo("***dsv2-flows*** overwrite called ") + logInfo("***dsv2-flows*** truncate called ") writeMode = SaveMode.Overwrite isTruncate = true this @@ -83,7 +84,7 @@ class JDBCWriteBuilder(options: JdbcOptionsInWrite, check filters. */ - logInfo("***dsv2-flows*** Overwrite table with new schema") + logInfo("***dsv2-flows*** processOverwrite called") false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md index c51795a7933c..a669735254cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md @@ -15,9 +15,20 @@ Status -> WIP ( Work in Progress), ReadyForReview, Done - Working branch is https://github.com/shivsood/spark-dsv2 - Intrested in contribution? Add work item and your name against it and party on. -## Major issues/mentions +## Issues/Questions/Mentions +- mode(overwrite) - what's the overwrite semantics in V2? + V1 overwrite will create a table if that does not exist. In V2 it seems that overwrite does not support + create semantics. overwrite with a non-existing table failed following tables:schema() which returned null + schema indicating no existing table. If not create scematics, then what's the diffrence between append + and overwrite without column filters. +- mode(overwrite) - why overwrite results in call to truncate(). Was expecting call to overwrite() instead. +- mode(append) - what's the append semantics in V2. + V1 append would throw an exception if the df col types are not same as table schema. Is that in v2 + achieved by Table::schema? schema returns source schema (as in db) and framework checks if the source schema is + compatible with the dataframe type. Tested diffirence is number of cols and framework would raise an exception , + but diffrence in data type did not raise any exception. - Lots of trivial logging. Draft implementation with API understanding as main goal -Update date : 7/19 +Update date : 7/22 From c2d270724a88f7c5c3d5aa8786a6480b3e0a6f37 Mon Sep 17 00:00:00 2001 From: shivsood Date: Tue, 23 Jul 2019 14:02:00 -0700 Subject: [PATCH 11/24] first draft of read implementation. No support for partitioning or filter as yet --- .../jdbc/MsSqlServerIntegrationSuite.scala | 10 +++ .../v2/jdbc/DBPartitionReader.scala | 67 +++++++++---------- .../v2/jdbc/DBPartitionReaderFactory.scala | 8 ++- .../datasources/v2/jdbc/DBTable.scala | 2 +- .../datasources/v2/jdbc/DBTableScan.scala | 29 ++++---- .../datasources/v2/jdbc/JDBCScanBuilder.scala | 21 +++--- 6 files changed, 71 insertions(+), 66 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index 3d8e68b6bb37..2ca10ab8575c 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -158,6 +158,16 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { df2.show() } + test("JDBCV2 read test") { + // Read table with JDBCV2 + val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable","strings_numbers").load() + val numberOfRows = df1.count + val df2 = spark.read.format("jdbcv2").option("url",jdbcUrl).option("dbtable","strings_numbers").load() + df2.show(10) + df2.select("i").show(10) + assert(df2.count == numberOfRows) + } + test("Basic test") { val df = spark.read.jdbc(jdbcUrl, "tbl", new Properties) val rows = df.collect() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala index 0fcc9f490815..59dda5645fb2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala @@ -19,57 +19,50 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import java.io.IOException +import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.sources.v2.reader.PartitionReader -import org.apache.spark.sql.types.{StringType, StructField, StructType} -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.sql.types.{StructType} -class DBPartitionReader(schema : StructType) extends PartitionReader[InternalRow] with Logging { - var dummyRows = 0 - /* - * Note : Read implementation is dummy as of now. - * It returns a hard coded schema and rows. - */ +/* + * Provides basic read implementation. + * TODO : multi executor paritition scenario + * TODO : Optimal JDBC connection parameters usage + */ +class DBPartitionReader(options: JDBCOptions, schema : StructType, + filters: Array[Filter], prunedCols: StructType) + extends PartitionReader[InternalRow] with Logging { + var retrievedRows = 0 + val sqlSelectStmtWithFilters = s"SELECT $prunedCols from ${options.tableOrQuery} $filters" + val sqlSelectStmt = s"SELECT * from ${options.tableOrQuery}" + val tc = TaskContext.get + val inputMetrics = tc.taskMetrics().inputMetrics + val conn = JdbcUtils.createConnectionFactory(options)() + val stmt = conn.prepareStatement(sqlSelectStmt) + val rs = stmt.executeQuery() + val itrRowIterator = JdbcUtils.resultSetToSparkInternalRows(rs, schema, inputMetrics) + + logInfo("***dsv2-flows*** DBPartitionReader created") + logInfo(s"***dsv2-flows*** DBPartitionReader SQL stmt $sqlSelectStmt") + logInfo(s"***dsv2-flows*** DBPartitionReader SQLWithFilters stmt is $sqlSelectStmtWithFilters") @throws[IOException] def next(): Boolean = { - logInfo("***dsv2-flows*** next() called") - if(dummyRows <2) { - dummyRows = dummyRows + 1 - true - } else { - false - } + itrRowIterator.hasNext } def get: InternalRow = { - - logInfo("***dsv2-flows*** get() called for row " + dummyRows) - - // Value for row1 - var v_name = "somename" - var v_rollnum = "38" - var v_occupation = "worker" - - if(dummyRows == 2) { - // Values for row2 - v_name = "someone" - v_rollnum = "39" - v_occupation = "manager" - } - - val values = schema.map(_.name).map { - case "name" => UTF8String.fromString(v_name) - case "rollnum" => UTF8String.fromString(v_rollnum) - case "occupation" => UTF8String.fromString(v_occupation) - case _ => UTF8String.fromString("anything") - } - InternalRow.fromSeq(values) + logInfo("***dsv2-flows*** get() called for row ") + retrievedRows = retrievedRows + 1 + itrRowIterator.next() } @throws[IOException] override def close(): Unit = { + logInfo(s"***dsv2-flows*** close called. number of rows retrieved is $retrievedRows") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala index ccf1e681544e..0ce3116d8636 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala @@ -19,12 +19,16 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} import org.apache.spark.sql.types.StructType -class DBPartitionReaderFactory(schema : StructType) extends PartitionReaderFactory with Logging{ +class DBPartitionReaderFactory(options: JDBCOptions, schema : StructType, + filters: Array[Filter], prunedCols: StructType) + extends PartitionReaderFactory with Logging{ def createReader(partition: InputPartition): PartitionReader[InternalRow] = { logInfo("***dsv2-flows*** createReader called") - new DBPartitionReader(schema) + new DBPartitionReader(options, schema, filters, prunedCols) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala index 296b120e030d..5cfa05c87ff7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -72,7 +72,7 @@ case class DBTable (sparkSession: SparkSession, override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { logInfo("***dsv2-flows*** newScanBuilder called") - new JDBCScanBuilder() + new JDBCScanBuilder(new JDBCOptions(options.asScala.toMap), userSchema) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala index cc757867e567..99fc8539d982 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala @@ -18,38 +18,37 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReaderFactory, Scan} -import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.sql.types.{StructType} -class DBTableScan extends Scan with Batch with Logging { - - /* - * Note : Read implementation is dummy as of now. - * It returns a hard coded schema and rows. - */ - - val table_schema = StructType(Seq( - StructField("name", StringType, true), - StructField("rollnum", StringType, true), - StructField("occupation", StringType, true))) +class DBTableScan(options: JDBCOptions, + filters: Array[Filter], + prunedCols: StructType) + extends Scan with Batch with Logging { + val conn = JdbcUtils.createConnectionFactory(options)() + val table_schema = JdbcUtils.getSchemaOption(conn, options) def readSchema: StructType = { logInfo("***dsv2-flows*** readSchema called") - table_schema - + table_schema.getOrElse(StructType(Nil)) } override def toBatch() : Batch = { + logInfo("***dsv2-flows*** toBatch()") this } def planInputPartitions: Array[InputPartition] = { + logInfo("***dsv2-flows*** planInputPartitions") Array(PartitionScheme) } def createReaderFactory: PartitionReaderFactory = { logInfo("***dsv2-flows*** createReaderFactory called") - new DBPartitionReaderFactory(table_schema) + new DBPartitionReaderFactory(options, table_schema.getOrElse(StructType(Nil)), + filters, prunedCols) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala index 6526840ebb22..73641ac035b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala @@ -18,37 +18,36 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.SupportsRead import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.types.StructType -class JDBCScanBuilder extends ScanBuilder with - SupportsPushDownFilters with SupportsPushDownRequiredColumns +class JDBCScanBuilder(options: JDBCOptions, + userSchema: Option[StructType]) + extends ScanBuilder with SupportsPushDownFilters with SupportsPushDownRequiredColumns with Logging { - var specifiedFilters: Array[Filter] = Array.empty + var prunedCols: StructType = _ def build: Scan = { logInfo("***dsv2-flows*** Scan called") - new DBTableScan() - + new DBTableScan(options, specifiedFilters, prunedCols) } def pushFilters(filters: Array[Filter]): Array[Filter] = { - logInfo("***dsv2-flows*** PushDown filters called") + logInfo(s"***dsv2-flows*** PushDown filters called filters as $filters") specifiedFilters = filters filters } def pruneColumns(requiredSchema: StructType): Unit = { - logInfo("***dsv2-flows*** pruneColumns called") - + logInfo(s"***dsv2-flows*** pruneColumns called with $requiredSchema") + prunedCols = requiredSchema } def pushedFilters: Array[Filter] = { - logInfo("***dsv2-flows*** pushedFilters called") + logInfo(s"***dsv2-flows*** pushedFilters called") specifiedFilters } - } From e15a9e6cb1828d144e0c446da7b18f1b71f78ab2 Mon Sep 17 00:00:00 2001 From: shivsood Date: Mon, 15 Jul 2019 21:30:35 -0700 Subject: [PATCH 12/24] scaffolding : basic read/write with jdbc connector --- examples/src/main/python/sql/datasourcev2.py | 20 +++++ ...pache.spark.sql.sources.DataSourceRegister | 1 + .../v2/jdbc/DBPartitionReader.scala | 76 ++++++++++++++++++ .../v2/jdbc/DBPartitionReaderFactory.scala | 32 ++++++++ .../datasources/v2/jdbc/DBTable.scala | 78 +++++++++++++++++++ .../datasources/v2/jdbc/DBTableScan.scala | 56 +++++++++++++ .../datasources/v2/jdbc/JDBCBatchWrite.scala | 42 ++++++++++ .../v2/jdbc/JDBCDataSourceV2.scala | 43 ++++++++++ .../datasources/v2/jdbc/JDBCDataWriter.scala | 56 +++++++++++++ .../v2/jdbc/JDBCDataWriterFactory.scala | 32 ++++++++ .../datasources/v2/jdbc/JDBCScanBuilder.scala | 55 +++++++++++++ .../v2/jdbc/JDBCWriteBuilder.scala | 40 ++++++++++ 12 files changed, 531 insertions(+) create mode 100644 examples/src/main/python/sql/datasourcev2.py create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala diff --git a/examples/src/main/python/sql/datasourcev2.py b/examples/src/main/python/sql/datasourcev2.py new file mode 100644 index 000000000000..392d17ae00ff --- /dev/null +++ b/examples/src/main/python/sql/datasourcev2.py @@ -0,0 +1,20 @@ +from pyspark.sql import SparkSession +from pyspark.sql.functions import col + +spark = SparkSession\ + .builder\ + .appName("DV2Test")\ + .getOrCreate() + + +df = spark.read.format("csv").options(header='true').load("/home/shivsood/myspark_scripts/text.csv") +df.show(5) + +#Read +df_jdbc = spark.read.options(header='true').format("jdbcv2").load() +df_jdbc.show(3) + + +#Write +#df.filter( col("rollnum") == "38" ).write.format("jdbcv2").save() + diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index c0b8b270bab1..5ce0de75ec46 100644 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -9,3 +9,4 @@ org.apache.spark.sql.execution.streaming.ConsoleSinkProvider org.apache.spark.sql.execution.streaming.sources.RateStreamProvider org.apache.spark.sql.execution.streaming.sources.TextSocketSourceProvider org.apache.spark.sql.execution.datasources.binaryfile.BinaryFileFormat +org.apache.spark.sql.execution.datasources.v2.jdbc.JDBCDataSourceV2 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala new file mode 100644 index 000000000000..9ce41711ca9b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala @@ -0,0 +1,76 @@ +/* + * 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.jdbc + +import java.io.IOException + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.unsafe.types.UTF8String + +class DBPartitionReader(schema : StructType) extends PartitionReader[InternalRow] with Logging { + + var dummyRows = 0 + + @throws[IOException] + def next(): Boolean = { + + logInfo("***dsv2-flows*** next() called") + + if(dummyRows <2) { + dummyRows = dummyRows + 1 + true + } else { + false + } + } + + def get: InternalRow = { + + logInfo("***dsv2-flows*** get() called for row " + dummyRows) + + // Value for row1 + var v_name = "shiv" + var v_rollnum = "38" + var v_occupation = "worker" + + if(dummyRows == 2) { + // Values for row2 + v_name = "someone" + v_rollnum = "39" + v_occupation = "dontknow" + } + + val values = schema.map(_.name).map { + case "name" => UTF8String.fromString(v_name) + case "rollnum" => UTF8String.fromString(v_rollnum) + case "occupation" => UTF8String.fromString(v_occupation) + case _ => UTF8String.fromString("anything") + } + + InternalRow.fromSeq(values) + } + + @throws[IOException] + override def close(): Unit = { + + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala new file mode 100644 index 000000000000..3c3d3a43e14a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala @@ -0,0 +1,32 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.types.StructType + +class DBPartitionReaderFactory(schema : StructType) extends PartitionReaderFactory with Logging{ + + def createReader(partition: InputPartition): PartitionReader[InternalRow] = { + logInfo("***dsv2-flows*** createReader called") + new DBPartitionReader(schema) + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala new file mode 100644 index 000000000000..478ceb9c28cf --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -0,0 +1,78 @@ +/* + * 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.jdbc + +import scala.collection.JavaConverters._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.v2.FileTable +import org.apache.spark.sql.execution.datasources.v2.csv.CSVWriteBuilder +import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.sources.v2.TableCapability.{BATCH_READ, BATCH_WRITE, TRUNCATE} +import org.apache.spark.sql.sources.v2.reader.ScanBuilder +import org.apache.spark.sql.sources.v2.writer.WriteBuilder +import org.apache.spark.sql.types.{DataType, IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class DBTable (sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + userSpecifiedSchema: Option[StructType]) + extends Table with SupportsWrite with SupportsRead with Logging{ + + + override def name: String = { + // TODO - Should come from user options + + logInfo("***dsv2-flows*** name called") + + "mysqltable" + } + + def schema: StructType = { + // TODO - Remove hardcoded schema + logInfo("***dsv2-flows*** schema called") + StructType(Seq( + StructField("name", StringType, true), + StructField("rollnum", StringType, true), + StructField("occupation", StringType, true))) + } + + override def capabilities: java.util.Set[TableCapability] = DBTable.CAPABILITIES + + def supportsDataType(dataType: DataType): Boolean = true + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + logInfo("***dsv2-flows*** newWriteBuilder called") + new JDBCWriteBuilder() + } + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + logInfo("***dsv2-flows*** newScanBuilder called") + new JDBCScanBuilder() + } + +} + +object DBTable { + private val CAPABILITIES = Set(BATCH_WRITE, BATCH_READ, TRUNCATE).asJava +} + + + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala new file mode 100644 index 000000000000..6b876dfc9429 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala @@ -0,0 +1,56 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReaderFactory, Scan} +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +class DBTableScan extends Scan with Batch with Logging { + + val table_schema = StructType(Seq( + StructField("name", StringType, true), + StructField("rollnum", StringType, true), + StructField("occupation", StringType, true))) + + def readSchema: StructType = { + logInfo("***dsv2-flows*** readSchema called") + table_schema + + } + + override def toBatch() : Batch = { + this + } + + def planInputPartitions: Array[InputPartition] = { + Array(PartitionScheme) + } + + def createReaderFactory: PartitionReaderFactory = { + + logInfo("***dsv2-flows*** createReaderFactory called") + new DBPartitionReaderFactory(table_schema) + + } + +} + +object PartitionScheme extends InputPartition { + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala new file mode 100644 index 000000000000..e4704b8bc737 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala @@ -0,0 +1,42 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, WriterCommitMessage} + +class JDBCBatchWrite extends BatchWrite with Logging{ + + def createBatchWriterFactory: DataWriterFactory = { + logInfo("***dsv2-flows*** createBatchWriterFactory called" ) + new JDBCDataWriterFactory() + } + + + def commit(messages: Array[WriterCommitMessage]): Unit = { + + logInfo("***dsv2-flows*** commit called with message... " ) + + } + + def abort(messages: Array[WriterCommitMessage]): Unit = { + logInfo("***dsv2-flows*** abort called with message... " ) + + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala new file mode 100644 index 000000000000..23ccc65ea082 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala @@ -0,0 +1,43 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.sources.v2.{Table, TableProvider} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class JDBCDataSourceV2 extends TableProvider with DataSourceRegister with Logging{ + + override def shortName(): String = { + logInfo("***dsv2-flows*** shortName - return connector name") + "jdbcv2" + } + + override def getTable(options: CaseInsensitiveStringMap): Table = { + logInfo("***dsv2-flows*** getTable called") + DBTable(SparkSession.active, options, None) + } + + override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { + logInfo("***dsv2-flows*** getTable called with schema") + DBTable(SparkSession.active, options, Some(schema)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala new file mode 100644 index 000000000000..91caf1a4058b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala @@ -0,0 +1,56 @@ +/* + * 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.jdbc + +import java.io.IOException + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage} +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +class JDBCDataWriter extends DataWriter[InternalRow] with Logging{ + + @throws[IOException] + def write(record: InternalRow): Unit = { + logInfo("***dsv2-flows*** write " ) + + val schema = StructType(Seq( + StructField("name", StringType, true), + StructField("rollnum", StringType, true), + StructField("occupation", StringType, true))) + + val ret = record.toSeq(schema) + logInfo("***dsv2-flows*** write " + ret.mkString(":") ) + } + + @throws[IOException] + def commit: WriterCommitMessage = { + logInfo("***dsv2-flows*** commit called " ) + JDBCWriterCommitMessage + } + + @throws[IOException] + def abort(): Unit = { + logInfo("***dsv2-flows*** abort called " ) + } +} + +object JDBCWriterCommitMessage extends WriterCommitMessage { + val commitMessage: String = "committed" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala new file mode 100644 index 000000000000..f8f99716228f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala @@ -0,0 +1,32 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory} + +class JDBCDataWriterFactory extends DataWriterFactory with Logging{ + + def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = { + logInfo("***dsv2-flows*** createWriter called " ) + + new JDBCDataWriter() + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala new file mode 100644 index 000000000000..ac927b70dde7 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala @@ -0,0 +1,55 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.v2.SupportsRead +import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.types.StructType + +class JDBCScanBuilder extends ScanBuilder with + SupportsPushDownFilters with SupportsPushDownRequiredColumns + with Logging { + + var specifiedFilters: Array[Filter] = Array.empty + + + def build: Scan = { + logInfo("***dsv2-flows*** Scan called") + new DBTableScan() + + } + + def pushFilters(filters: Array[Filter]): Array[Filter] = { + logInfo("***dsv2-flows*** PushDown filters called") + specifiedFilters = filters + filters + } + + def pruneColumns(requiredSchema: StructType): Unit = { + logInfo("***dsv2-flows*** pruneColumns called") + + } + + def pushedFilters: Array[Filter] = { + logInfo("***dsv2-flows*** pushedFilters called") + specifiedFilters + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala new file mode 100644 index 000000000000..bb50a965edaa --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala @@ -0,0 +1,40 @@ +/* + * 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.jdbc + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, WriteBuilder} +import org.apache.spark.sql.types.StructType + +class JDBCWriteBuilder extends WriteBuilder with Logging{ + + override def withQueryId(queryId: String): WriteBuilder = { + logInfo("***dsv2-flows*** withQueryId called with queryId" + queryId) + this + } + + override def withInputDataSchema(schema: StructType): WriteBuilder = { + logInfo("***dsv2-flows*** withInputDataSchema called with schema" + schema.printTreeString()) + this + } + + override def buildForBatch : BatchWrite = { + logInfo("***dsv2-flows*** BatchWrite called") + new JDBCBatchWrite() + } +} From 99f1a3b8a958f43efa2da60990bf636caa80d1a9 Mon Sep 17 00:00:00 2001 From: shivsood Date: Fri, 19 Jul 2019 15:45:39 -0700 Subject: [PATCH 13/24] first draft implemnentation of write(append) flow with datasourcev2. E2E test case added in MssQLServerIntegrationSuite --- .../jdbc/MsSqlServerIntegrationSuite.scala | 24 ++++++ .../datasources/jdbc/JdbcUtils.scala | 78 +++++++++++++++++ .../v2/jdbc/DBPartitionReaderFactory.scala | 1 - .../datasources/v2/jdbc/DBTable.scala | 52 ++++++------ .../datasources/v2/jdbc/JDBCBatchWrite.scala | 16 ++-- .../v2/jdbc/JDBCDataSourceV2.scala | 1 - .../datasources/v2/jdbc/JDBCDataWriter.scala | 22 ++--- .../v2/jdbc/JDBCDataWriterFactory.scala | 19 +++-- .../v2/jdbc/JDBCWriteBuilder.scala | 83 +++++++++++++++++-- .../execution/datasources/v2/jdbc/Utils.scala | 58 +++++++++++++ 10 files changed, 295 insertions(+), 59 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index efd7ca74c796..edc3cf0761e6 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -99,6 +99,30 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { |'the', 'lazy', |'dog') """.stripMargin).executeUpdate() + conn.prepareStatement( + """ + |CREATE TABLE strings_numbers ( + |i NVarChar(10), + |j INT, + |k NVarChar(20)) + """.stripMargin).executeUpdate() + conn.prepareStatement( + """ + |INSERT INTO strings_numbers VALUES ( + |'string',38, + |'big string') + """.stripMargin).executeUpdate() + } + + test("JDBCV2 write test") { + // Read 1 row using JDBC. Write(append) this row using jdbcv2. + val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() + df1.show(10) + assert(df1.count == 1) + df1.write.format("jdbcv2").mode("append").option("url",jdbcUrl).option("dbtable", "strings_numbers").save() + val df2 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() + df2.show(10) + assert(df2.count == 2) } test("Basic test") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 86a27b5afc25..63f62b31f89e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -870,4 +870,82 @@ object JdbcUtils extends Logging { statement.close() } } + + /** + * Inserts row into the database represented by the connection. + * + * @param conn - JDBCConnection to the given database. + * @param record - InternalRow to be inserted + * @throws IllegalArgumentException if the schema contains an unsupported type. + * Additions to JDBCUtils to support DSV2 JDBC adapter. + * TODO : Refactoring to remove makeIRSetter and use makeSetter with InternalRows + * TODO : Reactor to remove v2 imports/types from this pure DSV1 file. + * TODO : Support for Transactions, Isolation levels + * TODO : Just add to statement here. Commit should be outside of this function. + */ + def saveRow(conn: Connection, record: InternalRow, + options : JdbcOptionsInWrite, schema : StructType) : Unit = { + val dialect = JdbcDialects.get(options.url) + val rddSchema = JdbcUtils.getSchemaOption(conn, options) + val passedSchema = schema + + val insertStatement = JdbcUtils.getInsertStatement(options.table, + schema, rddSchema, true, dialect) + val stmt = conn.prepareStatement(insertStatement) + logInfo("***dsv2-flows*** insertStatement is $insertStatement") + val setters = schema.fields.map(f => makeIRSetter(conn, dialect, f.dataType)) + val nullTypes = schema.fields.map(f => getJdbcType(f.dataType, dialect).jdbcNullType) + val numFields = schema.fields.length + + var colNum = 0 + while (colNum < numFields) { + logInfo("***dsv2-flows*** stmt prep field " + colNum) + if (record.isNullAt(colNum)) { + stmt.setNull(colNum + 1, nullTypes(colNum)) + } else { + setters(colNum).apply(stmt, record, colNum) + } + colNum = colNum + 1 + } + stmt.execute() + } + + private type JDBCValueSetterIR = (PreparedStatement, InternalRow, Int) => Unit + + private def makeIRSetter(conn: Connection, dialect: JdbcDialect, + dataType: DataType): JDBCValueSetterIR = dataType match { + case IntegerType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setInt(pos + 1, row.getInt(pos)) + + case LongType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setLong(pos + 1, row.getLong(pos)) + + case DoubleType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setDouble(pos + 1, row.getDouble(pos)) + + case FloatType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setFloat(pos + 1, row.getFloat(pos)) + + case ShortType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setShort(pos + 1, row.getShort(pos)) + + case ByteType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setByte(pos + 1, row.getByte(pos)) + + case BooleanType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setBoolean(pos + 1, row.getBoolean(pos)) + + case StringType => + (stmt: PreparedStatement, row: InternalRow, pos: Int) => + stmt.setString(pos + 1, row.getString(pos)) + case _ => + throw new IllegalArgumentException(s"Not supported ${dataType.catalogString}") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala index 3c3d3a43e14a..03d86fa18229 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala @@ -28,5 +28,4 @@ class DBPartitionReaderFactory(schema : StructType) extends PartitionReaderFacto logInfo("***dsv2-flows*** createReader called") new DBPartitionReader(schema) } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala index 478ceb9c28cf..cea507454221 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -17,62 +17,60 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc +import java.sql.Connection + import scala.collection.JavaConverters._ import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.execution.datasources.v2.csv.CSVWriteBuilder +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions.JDBC_TABLE_NAME +import org.apache.spark.sql.execution.streaming.MetadataLogFileIndex import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, Table, TableCapability} -import org.apache.spark.sql.sources.v2.TableCapability.{BATCH_READ, BATCH_WRITE, TRUNCATE} +import org.apache.spark.sql.sources.v2.TableCapability.{BATCH_READ, BATCH_WRITE, OVERWRITE_BY_FILTER, TRUNCATE} import org.apache.spark.sql.sources.v2.reader.ScanBuilder import org.apache.spark.sql.sources.v2.writer.WriteBuilder import org.apache.spark.sql.types.{DataType, IntegerType, StringType, StructField, StructType} -import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql.util.{CaseInsensitiveStringMap, SchemaUtils} case class DBTable (sparkSession: SparkSession, - options: CaseInsensitiveStringMap, - userSpecifiedSchema: Option[StructType]) + options: CaseInsensitiveStringMap, + userSchema: Option[StructType]) extends Table with SupportsWrite with SupportsRead with Logging{ + private val userOptions = new JDBCOptions(options.asScala.toMap) + private val tableName = userOptions.parameters(JDBC_TABLE_NAME) + private val conn : Connection = JdbcUtils.createConnectionFactory(userOptions)() override def name: String = { - // TODO - Should come from user options - - logInfo("***dsv2-flows*** name called") - - "mysqltable" + logInfo("***dsv2-flows*** name called. Table name is " + tableName) + tableName } - def schema: StructType = { - // TODO - Remove hardcoded schema + override def schema: StructType = { + // TODO - check why a schema request? What if no table exists and + // no userSpecifiedSchema logInfo("***dsv2-flows*** schema called") - StructType(Seq( - StructField("name", StringType, true), - StructField("rollnum", StringType, true), - StructField("occupation", StringType, true))) + val schemaInDB = JdbcUtils.getSchemaOption(conn, userOptions) + Utils.logSchema("schema from DB", schemaInDB) + schemaInDB.getOrElse(StructType(Nil)) } override def capabilities: java.util.Set[TableCapability] = DBTable.CAPABILITIES - def supportsDataType(dataType: DataType): Boolean = true - override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { logInfo("***dsv2-flows*** newWriteBuilder called") - new JDBCWriteBuilder() + Utils.logSchema("Schema passed to DBTable", userSchema) + new JDBCWriteBuilder( + new JdbcOptionsInWrite(options.asScala.toMap), userSchema) } override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { logInfo("***dsv2-flows*** newScanBuilder called") new JDBCScanBuilder() } - } object DBTable { - private val CAPABILITIES = Set(BATCH_WRITE, BATCH_READ, TRUNCATE).asJava + private val CAPABILITIES = Set(BATCH_READ, BATCH_WRITE, TRUNCATE, OVERWRITE_BY_FILTER).asJava } - - - diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala index e4704b8bc737..952e05eb7530 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala @@ -18,25 +18,23 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.datasources.jdbc.JdbcOptionsInWrite import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.types.StructType -class JDBCBatchWrite extends BatchWrite with Logging{ +class JDBCBatchWrite(options: JdbcOptionsInWrite, fwPassedSchema: StructType) + extends BatchWrite with Logging{ def createBatchWriterFactory: DataWriterFactory = { logInfo("***dsv2-flows*** createBatchWriterFactory called" ) - new JDBCDataWriterFactory() + new JDBCDataWriterFactory(options, fwPassedSchema) } - - def commit(messages: Array[WriterCommitMessage]): Unit = { - - logInfo("***dsv2-flows*** commit called with message... " ) - + logInfo("***dsv2-flows*** commit called with $messages.length messages" + + s"with value as $messages.mkString(':')") } def abort(messages: Array[WriterCommitMessage]): Unit = { logInfo("***dsv2-flows*** abort called with message... " ) - } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala index 23ccc65ea082..aec0940b5bad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataSourceV2.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap class JDBCDataSourceV2 extends TableProvider with DataSourceRegister with Logging{ - override def shortName(): String = { logInfo("***dsv2-flows*** shortName - return connector name") "jdbcv2" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala index 91caf1a4058b..016921ca25c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala @@ -18,25 +18,24 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import java.io.IOException +import java.sql.{Connection, PreparedStatement} import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils._ +import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage} -import org.apache.spark.sql.types.{StringType, StructField, StructType} - -class JDBCDataWriter extends DataWriter[InternalRow] with Logging{ +import org.apache.spark.sql.types._ +class JDBCDataWriter(options: JdbcOptionsInWrite, + jdbcConn: Connection, schema: StructType) + extends DataWriter[InternalRow] with Logging{ @throws[IOException] def write(record: InternalRow): Unit = { logInfo("***dsv2-flows*** write " ) - - val schema = StructType(Seq( - StructField("name", StringType, true), - StructField("rollnum", StringType, true), - StructField("occupation", StringType, true))) - - val ret = record.toSeq(schema) - logInfo("***dsv2-flows*** write " + ret.mkString(":") ) + JdbcUtils.saveRow(jdbcConn, record, options, schema) } @throws[IOException] @@ -49,6 +48,7 @@ class JDBCDataWriter extends DataWriter[InternalRow] with Logging{ def abort(): Unit = { logInfo("***dsv2-flows*** abort called " ) } + } object JDBCWriterCommitMessage extends WriterCommitMessage { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala index f8f99716228f..db48840fba38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala @@ -17,16 +17,25 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc +import java.sql.Connection + import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory} +import org.apache.spark.sql.types.StructType -class JDBCDataWriterFactory extends DataWriterFactory with Logging{ - +/* Writer factory that's serialized and send to executors + */ +class JDBCDataWriterFactory(options: JdbcOptionsInWrite, schema: StructType) extends + DataWriterFactory with Logging{ def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = { - logInfo("***dsv2-flows*** createWriter called " ) - - new JDBCDataWriter() + // TODO : Check if every task in executor call createWriter to gets its own writer object. + // JDBC connection should not be shared between Tasks. + // JDBCUtil.createConnectionFactory should take care of that?? + logInfo(s"***dsv2-flows*** createWriter called for partition $partitionId taskID $taskId") + val conn: Connection = JdbcUtils.createConnectionFactory(options)() + new JDBCDataWriter(options, conn, schema) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala index bb50a965edaa..48b1348a7f73 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala @@ -17,24 +17,97 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc +import java.sql.{Connection, DriverManager} + import org.apache.spark.internal.Logging -import org.apache.spark.sql.sources.v2.writer.{BatchWrite, WriteBuilder} +import org.apache.spark.sql.SaveMode +import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.sources.{AlwaysTrue$, Filter} +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, SupportsOverwrite, WriteBuilder} import org.apache.spark.sql.types.StructType -class JDBCWriteBuilder extends WriteBuilder with Logging{ +class JDBCWriteBuilder(options: JdbcOptionsInWrite, + userSchema: Option[StructType]) + extends SupportsOverwrite with Logging{ + // TODO : Check, The default mode is assumed as Append. Refer physical plans to + // overwrite and append data i.e. OverwriteByExpressionExec and AppendDataExec + // respectively(Truncate and overwrite are called explicitly) + private var writeMode : SaveMode = SaveMode.Append + private var isTruncate : Boolean = false + private var fwPassedSchema : StructType = _ + private val conn : Connection = JdbcUtils.createConnectionFactory(options)() override def withQueryId(queryId: String): WriteBuilder = { logInfo("***dsv2-flows*** withQueryId called with queryId" + queryId) + // TODO : Check, Possible for his object to handles multiple queries on same table. this } override def withInputDataSchema(schema: StructType): WriteBuilder = { - logInfo("***dsv2-flows*** withInputDataSchema called with schema" + schema.printTreeString()) + logInfo("***dsv2-flows*** withInputDataSchema called with schema") + logInfo("***dsv2-flows*** schema is " + schema.printTreeString()) + fwPassedSchema = schema this } override def buildForBatch : BatchWrite = { - logInfo("***dsv2-flows*** BatchWrite called") - new JDBCBatchWrite() + logInfo("***dsv2-flows*** buildForBatch called") + writeMode match { + case SaveMode.Overwrite => + processOverwrite() + case SaveMode.Append => + processAppend() + } + new JDBCBatchWrite(options, fwPassedSchema) + } + + override def overwrite(filters: Array[Filter]): WriteBuilder = { + logInfo("***dsv2-flows*** overwrite called ") + writeMode = SaveMode.Overwrite + this + } + + override def truncate(): WriteBuilder = { + logInfo("***dsv2-flows*** overwrite called ") + writeMode = SaveMode.Overwrite + isTruncate = true + this + } + + def processOverwrite() : Boolean = { + /* Overwrite table logic + 1. Check if table exists. If not create it here. Should create be done?? + 2. If table exists and isTruncate, then just truncate existing table + 3. If table exists and !isTruncate, then recreate table with new schema + Post table creation, send requests to executors to insert data. + + check filters. + */ + logInfo("***dsv2-flows*** Overwrite table with new schema") + false + } + + def processAppend() : Unit = { + /* Append table logic + * 1. Check is table exists. Create if not. Step4. + * 2. If table exists and schema does not match, raise exception. + * 3. If table exists and schema match. Step4 + * 4. Send to executors for data insert + */ + logInfo("***dsv2-flows*** Append to table") + // log schemas received. + Utils.logSchema("userSchema", userSchema) + Utils.logSchema("fwPassedSchema", Option(fwPassedSchema)) + + JdbcUtils.tableExists(conn, options) match { + case true => + logInfo("***dsv2-flows*** Table exists" ) + Utils.strictSchemaCheck(fwPassedSchema) + logInfo("***dsv2-flows*** schema check done. Good to go." ) + case _ => + logInfo("***dsv2-flows*** Table does not exists." ) + // TODO : Check scemantics, Raise exception Or Create it. + Utils.createTable(fwPassedSchema) + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala new file mode 100644 index 000000000000..17909c3bd7b0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala @@ -0,0 +1,58 @@ +/* + * 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.jdbc + +import java.sql.{Connection, PreparedStatement} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils.{getCommonJDBCType, getJdbcType} +import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} +import org.apache.spark.sql.types.{BooleanType, ByteType, DataType, DoubleType, FloatType, IntegerType, LongType, ShortType, StringType, StructType} + +/* Misc utils + + */ +object Utils extends Logging{ + def logSchema(prefix: String, schema: Option[StructType]) : Unit = { + schema match { + case Some(i) => + val schemaString = i.printTreeString() + logInfo(s"***dsv2-flows*** $prefix schema exists" ) + logInfo(s"***dsv2-flows*** $prefix schemaInDB schema is $schemaString") + case None => + logInfo(s"***dsv2-flows*** $prefix schemaInDB schema is None" ) + } + } + + def createTable(structType: StructType): Unit = { + /* Create table per passed schema. Raise exception on any failure. + */ + } + + def strictSchemaCheck(schemaInSpark: StructType) : StructType = { + // TODO : Raise exception if fwPassedSchema is not same as schemaInDB. + schemaInSpark + } + + + + + +} From 9e4d2d629bb64b61cbc6ec7d5d6510675e141b49 Mon Sep 17 00:00:00 2001 From: shivsood Date: Fri, 19 Jul 2019 16:43:40 -0700 Subject: [PATCH 14/24] Readme added with high level project work items and plan --- .../execution/datasources/v2/jdbc/Readme.md | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md new file mode 100644 index 000000000000..c51795a7933c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md @@ -0,0 +1,23 @@ +# Plan/Status of ongoing work on DataSource V2 JDBC connector + +## Plan +| Work Item | Who's on it | Status | +|-----------------------------------------------| ----------- | ------ | +| Batch write ( append, overwrite, truncate) | shivsood | WIP | +| Streaming write | TBD | | +| Read path implementation | TBD | | +| Streaming read | TBD | | +| ?? | TBD | | + +Status -> WIP ( Work in Progress), ReadyForReview, Done + +## Others +- Working branch is https://github.com/shivsood/spark-dsv2 +- Intrested in contribution? Add work item and your name against it and party on. + +## Major issues/mentions +- Lots of trivial logging. Draft implementation with API understanding as main goal + +Update date : 7/19 + + From 72b949df7d364897b4de9247f5802d892684395a Mon Sep 17 00:00:00 2001 From: shivsood Date: Mon, 15 Jul 2019 21:30:35 -0700 Subject: [PATCH 15/24] scaffolding : basic read/write with jdbc connector --- .../datasources/v2/jdbc/DBPartitionReaderFactory.scala | 1 - .../spark/sql/execution/datasources/v2/jdbc/DBTable.scala | 5 +++-- .../sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala | 1 + .../sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala | 1 - .../datasources/v2/jdbc/JDBCDataWriterFactory.scala | 1 - 5 files changed, 4 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala index 03d86fa18229..ccf1e681544e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, import org.apache.spark.sql.types.StructType class DBPartitionReaderFactory(schema : StructType) extends PartitionReaderFactory with Logging{ - def createReader(partition: InputPartition): PartitionReader[InternalRow] = { logInfo("***dsv2-flows*** createReader called") new DBPartitionReader(schema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala index cea507454221..b898ae9740e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -58,13 +58,14 @@ case class DBTable (sparkSession: SparkSession, override def capabilities: java.util.Set[TableCapability] = DBTable.CAPABILITIES + def supportsDataType(dataType: DataType): Boolean = true + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { logInfo("***dsv2-flows*** newWriteBuilder called") Utils.logSchema("Schema passed to DBTable", userSchema) new JDBCWriteBuilder( new JdbcOptionsInWrite(options.asScala.toMap), userSchema) } - override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { logInfo("***dsv2-flows*** newScanBuilder called") new JDBCScanBuilder() @@ -72,5 +73,5 @@ case class DBTable (sparkSession: SparkSession, } object DBTable { - private val CAPABILITIES = Set(BATCH_READ, BATCH_WRITE, TRUNCATE, OVERWRITE_BY_FILTER).asJava + private val CAPABILITIES = Set(BATCH_WRITE, BATCH_READ, TRUNCATE).asJava } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala index 952e05eb7530..ce0bf312cb0d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import org.apache.spark.internal.Logging + import org.apache.spark.sql.execution.datasources.jdbc.JdbcOptionsInWrite import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, WriterCommitMessage} import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala index 016921ca25c2..b75d11d9c304 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala @@ -48,7 +48,6 @@ class JDBCDataWriter(options: JdbcOptionsInWrite, def abort(): Unit = { logInfo("***dsv2-flows*** abort called " ) } - } object JDBCWriterCommitMessage extends WriterCommitMessage { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala index db48840fba38..f9bb9f3827c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriterFactory.scala @@ -37,5 +37,4 @@ class JDBCDataWriterFactory(options: JdbcOptionsInWrite, schema: StructType) ext val conn: Connection = JdbcUtils.createConnectionFactory(options)() new JDBCDataWriter(options, conn, schema) } - } From f7b5e4d98cd83838bfe75594952a5a0fec54ed86 Mon Sep 17 00:00:00 2001 From: shivsood Date: Fri, 19 Jul 2019 15:45:39 -0700 Subject: [PATCH 16/24] first draft implemnentation of write(append) flow with datasourcev2. E2E test case added in MssQLServerIntegrationSuite --- .../spark/sql/execution/datasources/v2/jdbc/DBTable.scala | 4 +--- .../sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala | 1 - .../sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala | 1 + 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala index b898ae9740e8..983408c9951f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -58,8 +58,6 @@ case class DBTable (sparkSession: SparkSession, override def capabilities: java.util.Set[TableCapability] = DBTable.CAPABILITIES - def supportsDataType(dataType: DataType): Boolean = true - override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { logInfo("***dsv2-flows*** newWriteBuilder called") Utils.logSchema("Schema passed to DBTable", userSchema) @@ -73,5 +71,5 @@ case class DBTable (sparkSession: SparkSession, } object DBTable { - private val CAPABILITIES = Set(BATCH_WRITE, BATCH_READ, TRUNCATE).asJava + private val CAPABILITIES = Set(BATCH_READ, BATCH_WRITE, TRUNCATE, OVERWRITE_BY_FILTER).asJava } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala index ce0bf312cb0d..693de23ebc0b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.execution.datasources.jdbc.JdbcOptionsInWrite import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, WriterCommitMessage} import org.apache.spark.sql.types.StructType - class JDBCBatchWrite(options: JdbcOptionsInWrite, fwPassedSchema: StructType) extends BatchWrite with Logging{ def createBatchWriterFactory: DataWriterFactory = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala index b75d11d9c304..016921ca25c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCDataWriter.scala @@ -48,6 +48,7 @@ class JDBCDataWriter(options: JdbcOptionsInWrite, def abort(): Unit = { logInfo("***dsv2-flows*** abort called " ) } + } object JDBCWriterCommitMessage extends WriterCommitMessage { From 4a53903834f649072de2a0d507a43e6c35cdb710 Mon Sep 17 00:00:00 2001 From: shivsood Date: Sat, 20 Jul 2019 10:23:54 -0700 Subject: [PATCH 17/24] hygiene fixes and comments clarifing that read implementation is dummy for now --- .../datasources/v2/jdbc/DBPartitionReader.scala | 13 ++++++------- .../execution/datasources/v2/jdbc/DBTableScan.scala | 9 +++++---- .../datasources/v2/jdbc/JDBCScanBuilder.scala | 1 - .../sql/execution/datasources/v2/jdbc/Utils.scala | 5 ----- 4 files changed, 11 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala index 9ce41711ca9b..0fcc9f490815 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala @@ -26,14 +26,16 @@ import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.unsafe.types.UTF8String class DBPartitionReader(schema : StructType) extends PartitionReader[InternalRow] with Logging { - var dummyRows = 0 + /* + * Note : Read implementation is dummy as of now. + * It returns a hard coded schema and rows. + */ @throws[IOException] def next(): Boolean = { logInfo("***dsv2-flows*** next() called") - if(dummyRows <2) { dummyRows = dummyRows + 1 true @@ -47,7 +49,7 @@ class DBPartitionReader(schema : StructType) extends PartitionReader[InternalRow logInfo("***dsv2-flows*** get() called for row " + dummyRows) // Value for row1 - var v_name = "shiv" + var v_name = "somename" var v_rollnum = "38" var v_occupation = "worker" @@ -55,7 +57,7 @@ class DBPartitionReader(schema : StructType) extends PartitionReader[InternalRow // Values for row2 v_name = "someone" v_rollnum = "39" - v_occupation = "dontknow" + v_occupation = "manager" } val values = schema.map(_.name).map { @@ -64,13 +66,10 @@ class DBPartitionReader(schema : StructType) extends PartitionReader[InternalRow case "occupation" => UTF8String.fromString(v_occupation) case _ => UTF8String.fromString("anything") } - InternalRow.fromSeq(values) } @throws[IOException] override def close(): Unit = { - } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala index 6b876dfc9429..cc757867e567 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala @@ -23,6 +23,11 @@ import org.apache.spark.sql.types.{StringType, StructField, StructType} class DBTableScan extends Scan with Batch with Logging { + /* + * Note : Read implementation is dummy as of now. + * It returns a hard coded schema and rows. + */ + val table_schema = StructType(Seq( StructField("name", StringType, true), StructField("rollnum", StringType, true), @@ -43,14 +48,10 @@ class DBTableScan extends Scan with Batch with Logging { } def createReaderFactory: PartitionReaderFactory = { - logInfo("***dsv2-flows*** createReaderFactory called") new DBPartitionReaderFactory(table_schema) - } - } object PartitionScheme extends InputPartition { - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala index ac927b70dde7..6526840ebb22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala @@ -29,7 +29,6 @@ class JDBCScanBuilder extends ScanBuilder with var specifiedFilters: Array[Filter] = Array.empty - def build: Scan = { logInfo("***dsv2-flows*** Scan called") new DBTableScan() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala index 17909c3bd7b0..c43fb46f1e18 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala @@ -50,9 +50,4 @@ object Utils extends Logging{ // TODO : Raise exception if fwPassedSchema is not same as schemaInDB. schemaInSpark } - - - - - } From cc7af997390a9a7787172d93a63921a6743fa37e Mon Sep 17 00:00:00 2001 From: shivsood Date: Mon, 22 Jul 2019 14:41:56 -0700 Subject: [PATCH 18/24] cleaned up write(append) implementation. Append will not create a table and only append if a table exist. If table does not exist, a dbtable::schema request would return a null schema and fw will raise an exception. --- .../jdbc/MsSqlServerIntegrationSuite.scala | 26 ++++++++++++++++++- .../datasources/v2/jdbc/DBTable.scala | 6 +++-- .../v2/jdbc/JDBCWriteBuilder.scala | 18 ++----------- .../execution/datasources/v2/jdbc/Utils.scala | 12 +++++++-- 4 files changed, 41 insertions(+), 21 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index edc3cf0761e6..2f6f18171d3f 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -18,10 +18,12 @@ package org.apache.spark.sql.jdbc import java.math.BigDecimal -import java.sql.{Connection, Date, Timestamp} +import java.sql.{Connection, Date, Struct, Timestamp} import java.util.Properties import org.apache.spark.tags.DockerTest +import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.types._ @DockerTest class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { @@ -114,6 +116,20 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { """.stripMargin).executeUpdate() } + def create_test_df() : DataFrame = { + val schema:StructType = StructType( + Seq(StructField ("i", IntegerType, true), + StructField ("j", IntegerType, true), + StructField ("k", IntegerType, true)) + ) + val data:Seq[Row] = Seq( + Row(1,1,2), + Row(1,2,3) + ) + + spark.createDataFrame(spark.sparkContext.parallelize(data),schema) + } + test("JDBCV2 write test") { // Read 1 row using JDBC. Write(append) this row using jdbcv2. val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() @@ -123,6 +139,14 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { val df2 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() df2.show(10) assert(df2.count == 2) + + // Create a df with diffirent schema and append this to existing table. No convinced why this + // is passing. writing a dataframe with diffirent schema should fail. + val df_new = create_test_df() + df_new.write.format("jdbcv2").mode("append").option("url",jdbcUrl).option("dbtable", "strings_numbers").save() + val df2_new = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() + df2_new.show(10) + assert(df2_new.count == 4) } test("Basic test") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala index 983408c9951f..e4401b1b6d09 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -48,8 +48,10 @@ case class DBTable (sparkSession: SparkSession, } override def schema: StructType = { - // TODO - check why a schema request? What if no table exists and - // no userSpecifiedSchema + /* TODO - check why a schema request? Will this be called for every 'append' + * request to get schema from DBTable. If so, schema check in append should + * not be required. + */ logInfo("***dsv2-flows*** schema called") val schemaInDB = JdbcUtils.getSchemaOption(conn, userOptions) Utils.logSchema("schema from DB", schemaInDB) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala index 48b1348a7f73..52174a4e96fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala @@ -88,26 +88,12 @@ class JDBCWriteBuilder(options: JdbcOptionsInWrite, } def processAppend() : Unit = { - /* Append table logic - * 1. Check is table exists. Create if not. Step4. - * 2. If table exists and schema does not match, raise exception. - * 3. If table exists and schema match. Step4 - * 4. Send to executors for data insert + /* Append table logic : If we have reached this far, table exist and schema check is done. + * So processappend does nothing here. just sends request to executors for data insert */ logInfo("***dsv2-flows*** Append to table") // log schemas received. Utils.logSchema("userSchema", userSchema) Utils.logSchema("fwPassedSchema", Option(fwPassedSchema)) - - JdbcUtils.tableExists(conn, options) match { - case true => - logInfo("***dsv2-flows*** Table exists" ) - Utils.strictSchemaCheck(fwPassedSchema) - logInfo("***dsv2-flows*** schema check done. Good to go." ) - case _ => - logInfo("***dsv2-flows*** Table does not exists." ) - // TODO : Check scemantics, Raise exception Or Create it. - Utils.createTable(fwPassedSchema) - } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala index c43fb46f1e18..2c64093a2045 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import java.sql.{Connection, PreparedStatement} import org.apache.spark.internal.Logging +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils.{getCommonJDBCType, getJdbcType} @@ -46,8 +47,15 @@ object Utils extends Logging{ */ } - def strictSchemaCheck(schemaInSpark: StructType) : StructType = { + def strictSchemaCheck(schemaInSpark: StructType, dbTableSchema: StructType) : Boolean = { // TODO : Raise exception if fwPassedSchema is not same as schemaInDB. - schemaInSpark + if (schemaInSpark == dbTableSchema) { + logInfo(s"***dsv2-flows*** strictSchemaCheck passed" ) + true + } else { + logInfo(s"***dsv2-flows*** schema check failed" ) + throw new AnalysisException( + s"Schema does not match with that with the database table") + } } } From 247c72dce2d318bf0615b89a5010ccbc275df5bc Mon Sep 17 00:00:00 2001 From: shivsood Date: Mon, 22 Jul 2019 16:23:52 -0700 Subject: [PATCH 19/24] comments/questions on overwrite scematics and minor log fixes --- .../sql/jdbc/MsSqlServerIntegrationSuite.scala | 11 ++++++++++- .../execution/datasources/v2/jdbc/DBTable.scala | 9 ++++++--- .../datasources/v2/jdbc/JDBCWriteBuilder.scala | 5 +++-- .../sql/execution/datasources/v2/jdbc/Readme.md | 15 +++++++++++++-- 4 files changed, 32 insertions(+), 8 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index 2f6f18171d3f..3d8e68b6bb37 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -130,7 +130,7 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { spark.createDataFrame(spark.sparkContext.parallelize(data),schema) } - test("JDBCV2 write test") { + test("JDBCV2 write append test") { // Read 1 row using JDBC. Write(append) this row using jdbcv2. val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() df1.show(10) @@ -149,6 +149,15 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { assert(df2_new.count == 4) } + test("JDBCV2 write overwrite test") { + // Overwrite a existing table with a new schema and values. + val df1 = create_test_df() + // Overwrite test. Overwrite mode create a new table if it does not exist + df1.write.format("jdbcv2").mode("overwrite").option("url",jdbcUrl).option("dbtable","strings_numbers").save() + val df2 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable","strings_numbers").load() + df2.show() + } + test("Basic test") { val df = spark.read.jdbc(jdbcUrl, "tbl", new Properties) val rows = df.collect() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala index e4401b1b6d09..225c12a56613 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -48,9 +48,12 @@ case class DBTable (sparkSession: SparkSession, } override def schema: StructType = { - /* TODO - check why a schema request? Will this be called for every 'append' - * request to get schema from DBTable. If so, schema check in append should - * not be required. + /* TODO - check why a schema request? + * 1. Will this be called for every 'append' request to get schema from DBTable. + * If so, schema check in append should not be required. + * 2. This is called in overwrite as well. What if the table does not exist. A empty + * schema would be returned resulting in an exception. Should Overwrite semantics + * not create a table. */ logInfo("***dsv2-flows*** schema called") val schemaInDB = JdbcUtils.getSchemaOption(conn, userOptions) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala index 52174a4e96fb..2dd141df70d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala @@ -64,11 +64,12 @@ class JDBCWriteBuilder(options: JdbcOptionsInWrite, override def overwrite(filters: Array[Filter]): WriteBuilder = { logInfo("***dsv2-flows*** overwrite called ") writeMode = SaveMode.Overwrite + for(filter <- filters) logInfo(s"***dsv2-flows*** overwrite filter is $filter") this } override def truncate(): WriteBuilder = { - logInfo("***dsv2-flows*** overwrite called ") + logInfo("***dsv2-flows*** truncate called ") writeMode = SaveMode.Overwrite isTruncate = true this @@ -83,7 +84,7 @@ class JDBCWriteBuilder(options: JdbcOptionsInWrite, check filters. */ - logInfo("***dsv2-flows*** Overwrite table with new schema") + logInfo("***dsv2-flows*** processOverwrite called") false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md index c51795a7933c..a669735254cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md @@ -15,9 +15,20 @@ Status -> WIP ( Work in Progress), ReadyForReview, Done - Working branch is https://github.com/shivsood/spark-dsv2 - Intrested in contribution? Add work item and your name against it and party on. -## Major issues/mentions +## Issues/Questions/Mentions +- mode(overwrite) - what's the overwrite semantics in V2? + V1 overwrite will create a table if that does not exist. In V2 it seems that overwrite does not support + create semantics. overwrite with a non-existing table failed following tables:schema() which returned null + schema indicating no existing table. If not create scematics, then what's the diffrence between append + and overwrite without column filters. +- mode(overwrite) - why overwrite results in call to truncate(). Was expecting call to overwrite() instead. +- mode(append) - what's the append semantics in V2. + V1 append would throw an exception if the df col types are not same as table schema. Is that in v2 + achieved by Table::schema? schema returns source schema (as in db) and framework checks if the source schema is + compatible with the dataframe type. Tested diffirence is number of cols and framework would raise an exception , + but diffrence in data type did not raise any exception. - Lots of trivial logging. Draft implementation with API understanding as main goal -Update date : 7/19 +Update date : 7/22 From 508dbc4416bce571cd59a8f085e5f6d1a163d204 Mon Sep 17 00:00:00 2001 From: shivsood Date: Tue, 23 Jul 2019 14:02:00 -0700 Subject: [PATCH 20/24] first draft of read implementation. No support for partitioning or filter as yet --- .../jdbc/MsSqlServerIntegrationSuite.scala | 10 +++ .../v2/jdbc/DBPartitionReader.scala | 67 +++++++++---------- .../v2/jdbc/DBPartitionReaderFactory.scala | 8 ++- .../datasources/v2/jdbc/DBTable.scala | 2 +- .../datasources/v2/jdbc/DBTableScan.scala | 29 ++++---- .../datasources/v2/jdbc/JDBCScanBuilder.scala | 21 +++--- 6 files changed, 71 insertions(+), 66 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index 3d8e68b6bb37..2ca10ab8575c 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -158,6 +158,16 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { df2.show() } + test("JDBCV2 read test") { + // Read table with JDBCV2 + val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable","strings_numbers").load() + val numberOfRows = df1.count + val df2 = spark.read.format("jdbcv2").option("url",jdbcUrl).option("dbtable","strings_numbers").load() + df2.show(10) + df2.select("i").show(10) + assert(df2.count == numberOfRows) + } + test("Basic test") { val df = spark.read.jdbc(jdbcUrl, "tbl", new Properties) val rows = df.collect() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala index 0fcc9f490815..59dda5645fb2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala @@ -19,57 +19,50 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import java.io.IOException +import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.sources.v2.reader.PartitionReader -import org.apache.spark.sql.types.{StringType, StructField, StructType} -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.sql.types.{StructType} -class DBPartitionReader(schema : StructType) extends PartitionReader[InternalRow] with Logging { - var dummyRows = 0 - /* - * Note : Read implementation is dummy as of now. - * It returns a hard coded schema and rows. - */ +/* + * Provides basic read implementation. + * TODO : multi executor paritition scenario + * TODO : Optimal JDBC connection parameters usage + */ +class DBPartitionReader(options: JDBCOptions, schema : StructType, + filters: Array[Filter], prunedCols: StructType) + extends PartitionReader[InternalRow] with Logging { + var retrievedRows = 0 + val sqlSelectStmtWithFilters = s"SELECT $prunedCols from ${options.tableOrQuery} $filters" + val sqlSelectStmt = s"SELECT * from ${options.tableOrQuery}" + val tc = TaskContext.get + val inputMetrics = tc.taskMetrics().inputMetrics + val conn = JdbcUtils.createConnectionFactory(options)() + val stmt = conn.prepareStatement(sqlSelectStmt) + val rs = stmt.executeQuery() + val itrRowIterator = JdbcUtils.resultSetToSparkInternalRows(rs, schema, inputMetrics) + + logInfo("***dsv2-flows*** DBPartitionReader created") + logInfo(s"***dsv2-flows*** DBPartitionReader SQL stmt $sqlSelectStmt") + logInfo(s"***dsv2-flows*** DBPartitionReader SQLWithFilters stmt is $sqlSelectStmtWithFilters") @throws[IOException] def next(): Boolean = { - logInfo("***dsv2-flows*** next() called") - if(dummyRows <2) { - dummyRows = dummyRows + 1 - true - } else { - false - } + itrRowIterator.hasNext } def get: InternalRow = { - - logInfo("***dsv2-flows*** get() called for row " + dummyRows) - - // Value for row1 - var v_name = "somename" - var v_rollnum = "38" - var v_occupation = "worker" - - if(dummyRows == 2) { - // Values for row2 - v_name = "someone" - v_rollnum = "39" - v_occupation = "manager" - } - - val values = schema.map(_.name).map { - case "name" => UTF8String.fromString(v_name) - case "rollnum" => UTF8String.fromString(v_rollnum) - case "occupation" => UTF8String.fromString(v_occupation) - case _ => UTF8String.fromString("anything") - } - InternalRow.fromSeq(values) + logInfo("***dsv2-flows*** get() called for row ") + retrievedRows = retrievedRows + 1 + itrRowIterator.next() } @throws[IOException] override def close(): Unit = { + logInfo(s"***dsv2-flows*** close called. number of rows retrieved is $retrievedRows") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala index ccf1e681544e..0ce3116d8636 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReaderFactory.scala @@ -19,12 +19,16 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} import org.apache.spark.sql.types.StructType -class DBPartitionReaderFactory(schema : StructType) extends PartitionReaderFactory with Logging{ +class DBPartitionReaderFactory(options: JDBCOptions, schema : StructType, + filters: Array[Filter], prunedCols: StructType) + extends PartitionReaderFactory with Logging{ def createReader(partition: InputPartition): PartitionReader[InternalRow] = { logInfo("***dsv2-flows*** createReader called") - new DBPartitionReader(schema) + new DBPartitionReader(options, schema, filters, prunedCols) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala index 225c12a56613..b6dc093f6088 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -71,7 +71,7 @@ case class DBTable (sparkSession: SparkSession, } override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { logInfo("***dsv2-flows*** newScanBuilder called") - new JDBCScanBuilder() + new JDBCScanBuilder(new JDBCOptions(options.asScala.toMap), userSchema) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala index cc757867e567..99fc8539d982 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala @@ -18,38 +18,37 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReaderFactory, Scan} -import org.apache.spark.sql.types.{StringType, StructField, StructType} +import org.apache.spark.sql.types.{StructType} -class DBTableScan extends Scan with Batch with Logging { - - /* - * Note : Read implementation is dummy as of now. - * It returns a hard coded schema and rows. - */ - - val table_schema = StructType(Seq( - StructField("name", StringType, true), - StructField("rollnum", StringType, true), - StructField("occupation", StringType, true))) +class DBTableScan(options: JDBCOptions, + filters: Array[Filter], + prunedCols: StructType) + extends Scan with Batch with Logging { + val conn = JdbcUtils.createConnectionFactory(options)() + val table_schema = JdbcUtils.getSchemaOption(conn, options) def readSchema: StructType = { logInfo("***dsv2-flows*** readSchema called") - table_schema - + table_schema.getOrElse(StructType(Nil)) } override def toBatch() : Batch = { + logInfo("***dsv2-flows*** toBatch()") this } def planInputPartitions: Array[InputPartition] = { + logInfo("***dsv2-flows*** planInputPartitions") Array(PartitionScheme) } def createReaderFactory: PartitionReaderFactory = { logInfo("***dsv2-flows*** createReaderFactory called") - new DBPartitionReaderFactory(table_schema) + new DBPartitionReaderFactory(options, table_schema.getOrElse(StructType(Nil)), + filters, prunedCols) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala index 6526840ebb22..73641ac035b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCScanBuilder.scala @@ -18,37 +18,36 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.v2.SupportsRead import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.types.StructType -class JDBCScanBuilder extends ScanBuilder with - SupportsPushDownFilters with SupportsPushDownRequiredColumns +class JDBCScanBuilder(options: JDBCOptions, + userSchema: Option[StructType]) + extends ScanBuilder with SupportsPushDownFilters with SupportsPushDownRequiredColumns with Logging { - var specifiedFilters: Array[Filter] = Array.empty + var prunedCols: StructType = _ def build: Scan = { logInfo("***dsv2-flows*** Scan called") - new DBTableScan() - + new DBTableScan(options, specifiedFilters, prunedCols) } def pushFilters(filters: Array[Filter]): Array[Filter] = { - logInfo("***dsv2-flows*** PushDown filters called") + logInfo(s"***dsv2-flows*** PushDown filters called filters as $filters") specifiedFilters = filters filters } def pruneColumns(requiredSchema: StructType): Unit = { - logInfo("***dsv2-flows*** pruneColumns called") - + logInfo(s"***dsv2-flows*** pruneColumns called with $requiredSchema") + prunedCols = requiredSchema } def pushedFilters: Array[Filter] = { - logInfo("***dsv2-flows*** pushedFilters called") + logInfo(s"***dsv2-flows*** pushedFilters called") specifiedFilters } - } From 9d4093fefb4bc09f4eb9dd1f15aa3aa6c3b4d9c5 Mon Sep 17 00:00:00 2001 From: shivsood Date: Thu, 25 Jul 2019 16:12:32 -0700 Subject: [PATCH 21/24] cleaner more stuctured read implementation --- .../jdbc/MsSqlServerIntegrationSuite.scala | 45 ++++++++-------- .../v2/jdbc/DBPartitionReader.scala | 17 ++---- .../datasources/v2/jdbc/JDBCBatchWrite.scala | 1 - .../execution/datasources/v2/jdbc/Readme.md | 2 +- .../execution/datasources/v2/jdbc/Utils.scala | 54 ++++++++++++++++++- 5 files changed, 81 insertions(+), 38 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index 2ca10ab8575c..f2df6c7905e1 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -103,16 +103,17 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { """.stripMargin).executeUpdate() conn.prepareStatement( """ - |CREATE TABLE strings_numbers ( - |i NVarChar(10), + |CREATE TABLE dsv2testTbl ( + |i NVarChar(20), |j INT, |k NVarChar(20)) """.stripMargin).executeUpdate() conn.prepareStatement( """ - |INSERT INTO strings_numbers VALUES ( - |'string',38, - |'big string') + |INSERT INTO dsv2testTbl VALUES ( + |'The number', + |1, + |'Rocks!') """.stripMargin).executeUpdate() } @@ -130,21 +131,31 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { spark.createDataFrame(spark.sparkContext.parallelize(data),schema) } + test("JDBCV2 read test") { + // Read table with JDBCV2 + val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable","dsv2testTbl").load() + val numberOfRows = df1.count + df1.show(10) + val df2 = spark.read.format("jdbcv2").option("url",jdbcUrl).option("dbtable","dsv2testTbl").load() + df2.show(10) + assert(df2.count == numberOfRows) + } + test("JDBCV2 write append test") { // Read 1 row using JDBC. Write(append) this row using jdbcv2. - val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() + val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "dsv2testTbl").load() df1.show(10) assert(df1.count == 1) - df1.write.format("jdbcv2").mode("append").option("url",jdbcUrl).option("dbtable", "strings_numbers").save() - val df2 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() + df1.write.format("jdbcv2").mode("append").option("url",jdbcUrl).option("dbtable", "dsv2testTbl").save() + val df2 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "dsv2testTbl").load() df2.show(10) assert(df2.count == 2) // Create a df with diffirent schema and append this to existing table. No convinced why this // is passing. writing a dataframe with diffirent schema should fail. val df_new = create_test_df() - df_new.write.format("jdbcv2").mode("append").option("url",jdbcUrl).option("dbtable", "strings_numbers").save() - val df2_new = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "strings_numbers").load() + df_new.write.format("jdbcv2").mode("append").option("url",jdbcUrl).option("dbtable", "dsv2testTbl").save() + val df2_new = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "dsv2testTbl").load() df2_new.show(10) assert(df2_new.count == 4) } @@ -153,21 +164,11 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { // Overwrite a existing table with a new schema and values. val df1 = create_test_df() // Overwrite test. Overwrite mode create a new table if it does not exist - df1.write.format("jdbcv2").mode("overwrite").option("url",jdbcUrl).option("dbtable","strings_numbers").save() - val df2 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable","strings_numbers").load() + df1.write.format("jdbcv2").mode("overwrite").option("url",jdbcUrl).option("dbtable","dsv2testTbl").save() + val df2 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable","dsv2testTbl").load() df2.show() } - test("JDBCV2 read test") { - // Read table with JDBCV2 - val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable","strings_numbers").load() - val numberOfRows = df1.count - val df2 = spark.read.format("jdbcv2").option("url",jdbcUrl).option("dbtable","strings_numbers").load() - df2.show(10) - df2.select("i").show(10) - assert(df2.count == numberOfRows) - } - test("Basic test") { val df = spark.read.jdbc(jdbcUrl, "tbl", new Properties) val rows = df.collect() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala index 59dda5645fb2..a0c07e074da9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBPartitionReader.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import java.io.IOException +import java.sql.ResultSet import org.apache.spark.TaskContext import org.apache.spark.internal.Logging @@ -25,7 +26,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.sources.v2.reader.PartitionReader -import org.apache.spark.sql.types.{StructType} +import org.apache.spark.sql.types.StructType /* * Provides basic read implementation. @@ -36,18 +37,10 @@ class DBPartitionReader(options: JDBCOptions, schema : StructType, filters: Array[Filter], prunedCols: StructType) extends PartitionReader[InternalRow] with Logging { var retrievedRows = 0 - val sqlSelectStmtWithFilters = s"SELECT $prunedCols from ${options.tableOrQuery} $filters" - val sqlSelectStmt = s"SELECT * from ${options.tableOrQuery}" - val tc = TaskContext.get - val inputMetrics = tc.taskMetrics().inputMetrics - val conn = JdbcUtils.createConnectionFactory(options)() - val stmt = conn.prepareStatement(sqlSelectStmt) - val rs = stmt.executeQuery() - val itrRowIterator = JdbcUtils.resultSetToSparkInternalRows(rs, schema, inputMetrics) - logInfo("***dsv2-flows*** DBPartitionReader created") - logInfo(s"***dsv2-flows*** DBPartitionReader SQL stmt $sqlSelectStmt") - logInfo(s"***dsv2-flows*** DBPartitionReader SQLWithFilters stmt is $sqlSelectStmtWithFilters") + logInfo(s"***dsv2-flows*** passed pruned cols : $prunedCols and filters : $filters") + + var itrRowIterator = Utils.executeSelect(options, prunedCols, filters, schema) @throws[IOException] def next(): Boolean = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala index 693de23ebc0b..cdf205edcf02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCBatchWrite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc import org.apache.spark.internal.Logging - import org.apache.spark.sql.execution.datasources.jdbc.JdbcOptionsInWrite import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, WriterCommitMessage} import org.apache.spark.sql.types.StructType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md index a669735254cc..64e8d981baa9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md @@ -5,7 +5,7 @@ |-----------------------------------------------| ----------- | ------ | | Batch write ( append, overwrite, truncate) | shivsood | WIP | | Streaming write | TBD | | -| Read path implementation | TBD | | +| Read path implementation | shivsood | WIP | | Streaming read | TBD | | | ?? | TBD | | diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala index 2c64093a2045..88103ca0ec6e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala @@ -17,14 +17,16 @@ package org.apache.spark.sql.execution.datasources.v2.jdbc -import java.sql.{Connection, PreparedStatement} +import java.sql.{Connection, PreparedStatement, ResultSet} +import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.jdbc.{JdbcOptionsInWrite, JdbcUtils} +import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcOptionsInWrite, JdbcUtils} import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils.{getCommonJDBCType, getJdbcType} import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects, JdbcType} +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.{BooleanType, ByteType, DataType, DoubleType, FloatType, IntegerType, LongType, ShortType, StringType, StructType} /* Misc utils @@ -58,4 +60,52 @@ object Utils extends Logging{ s"Schema does not match with that with the database table") } } + def colList(cols: StructType): String = { + // TODO: column names in quotes + cols match { + case null => + logInfo(s"***dsv2-flows*** prunedCols is NULL") + logInfo(s"***dsv2-flows*** returning * ") + "*" + case _ => + logInfo(s"***dsv2-flows*** prunedCols is NULL") + val colArr = cols.names + colArr.length match { + case 0 => + logInfo(s"***dsv2-flows*** returning *") + "*" + case _ => colArr.mkString(",") + } + } + } + + def filterList(filters: Array[Filter]) : String = { + // TODO: Support for filters + "" + } + + def createSelectStmt(prunedCols: StructType, + filters: Array[Filter], + table: String): String = { + val cols = colList(prunedCols) + val filtersToAdd = filterList(filters) + val selectStmt = s"SELECT $cols FROM $table $filtersToAdd" + logInfo(s"***dsv2-flows*** selectStmt is $selectStmt") + selectStmt + } + + def executeSelect(options: JDBCOptions, prunedCols: StructType, + filters: Array[Filter], schema : StructType): Iterator[InternalRow] = { + val stmtString = createSelectStmt(prunedCols, filters, options.tableOrQuery) + logInfo(s"***dsv2-flows*** executeStmt is $stmtString") + // TODO : Why we do need the inputMetric here. + val inputMetrics = TaskContext.get.taskMetrics().inputMetrics + val conn = JdbcUtils.createConnectionFactory(options)() + val stmt = conn.prepareStatement(stmtString, + ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY) + stmt.setFetchSize(options.fetchSize) + stmt.setQueryTimeout(options.queryTimeout) + val rs = stmt.executeQuery() + JdbcUtils.resultSetToSparkInternalRows(rs, schema, inputMetrics) + } } From 1446faad993286a59d59c05462cc782c1e777528 Mon Sep 17 00:00:00 2001 From: shivsood Date: Wed, 31 Jul 2019 12:50:59 -0700 Subject: [PATCH 22/24] Fixed for OverWrite(with truncate). Test added in MsSqlServerIntegrationSuite.scala Scemantics are TRUNCATE TABLE and then overwrite with new data. Existing table schema is preserved. Overwrite(w/o truncate) - Scaffolding in place. Utils::CreateTable is dummy. Still need to be implemented. Scemantics are DROP TABLE, CREATE TABLE with new passed schema and then overwrite with new data. Problems - FW keep calling WriteBuilder::truncate() even when the truncate option is not specified or truncate explicitly set to false. Test update with truncate=false. - Added test df.filter and then overwrite(w/o) truncate to only write set of rows that match filter. FW still calls truncate Read path fixed to return schema with pruned columns as suggsted in Scan::readSchema Select with pruned columns still does not work. --- .../jdbc/MsSqlServerIntegrationSuite.scala | 113 +++++++++++++++--- .../datasources/v2/jdbc/DBTable.scala | 6 +- .../datasources/v2/jdbc/DBTableScan.scala | 2 +- .../v2/jdbc/JDBCWriteBuilder.scala | 22 ++-- .../execution/datasources/v2/jdbc/Readme.md | 71 ++++++++--- .../execution/datasources/v2/jdbc/Utils.scala | 3 +- 6 files changed, 170 insertions(+), 47 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index f2df6c7905e1..e0b376a99d05 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -132,41 +132,124 @@ class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { } test("JDBCV2 read test") { - // Read table with JDBCV2 - val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable","dsv2testTbl").load() + // Read table with JDBC and JDBCV2. Check that same row counts are returned. + val df1 = spark.read + .format("jdbc") + .option("url",jdbcUrl) + .option("dbtable","dsv2testTbl") + .load() val numberOfRows = df1.count df1.show(10) - val df2 = spark.read.format("jdbcv2").option("url",jdbcUrl).option("dbtable","dsv2testTbl").load() + + val df2 = spark.read + .format("jdbcv2") + .option("url",jdbcUrl) + .option("dbtable","dsv2testTbl") + .load() df2.show(10) + // Try column pruning. This fails with error. + //df2.select("i").show(10) assert(df2.count == numberOfRows) } test("JDBCV2 write append test") { - // Read 1 row using JDBC. Write(append) this row using jdbcv2. - val df1 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "dsv2testTbl").load() + // Read 1 row using JDBC. Write(append) this row using JDBCV2. + logInfo(s"***dsv2-flow-test*** Overwrite(append) can append to table with same schema") + val df1 = spark.read + .format("jdbc") + .option("url",jdbcUrl) + .option("dbtable", "dsv2testTbl") + .load() df1.show(10) assert(df1.count == 1) - df1.write.format("jdbcv2").mode("append").option("url",jdbcUrl).option("dbtable", "dsv2testTbl").save() - val df2 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "dsv2testTbl").load() + + df1.write.format("jdbcv2") + .mode("append") + .option("url",jdbcUrl) + .option("dbtable", "dsv2testTbl") + .save() + val df2 = spark.read + .format("jdbc") + .option("url",jdbcUrl) + .option("dbtable", "dsv2testTbl") + .load() df2.show(10) assert(df2.count == 2) - // Create a df with diffirent schema and append this to existing table. No convinced why this - // is passing. writing a dataframe with diffirent schema should fail. + // Create a df with diffirent schema and append existing table with a diffirent schema. Should fail. + // TODO : Writing a dataframe with diffirent schema should fail. Why is this passing. + logInfo(s"***dsv2-flow-test*** Overwrite(append) fails writing to table with diffirent schema") val df_new = create_test_df() - df_new.write.format("jdbcv2").mode("append").option("url",jdbcUrl).option("dbtable", "dsv2testTbl").save() - val df2_new = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable", "dsv2testTbl").load() + df_new.write + .format("jdbcv2") + .mode("append") + .option("url", jdbcUrl) + .option("dbtable", "dsv2testTbl") + .save() + val df2_new = spark.read + .format("jdbc") + .option("url", jdbcUrl) + .option("dbtable", "dsv2testTbl") + .load() df2_new.show(10) assert(df2_new.count == 4) } + test("JDBCV2 write overwrite(truncate) test") { + // Overwrite with truncate - Check orignal table schema is retined. + val df_ori_before = spark + .read + .format("jdbc") + .option("url",jdbcUrl) + .option("dbtable","dsv2testTbl") + .load() + df_ori_before.show() + val cl_types_orignal = df_ori_before.schema.fields.map(f=>f.dataType) + val t_before = cl_types_orignal.mkString(":") + logInfo(s"***dsv2-flow-test*** Column type before was $t_before") + + val df_new = create_test_df() + df_new.write + .format("jdbcv2") + .mode("overwrite") + .option("url",jdbcUrl) + .option("truncate","true") + .option("dbtable","dsv2testTbl") + .save() + + val df_ori_after = spark + .read + .format("jdbc") + .option("url",jdbcUrl) + .option("dbtable","dsv2testTbl") + .load() + df_ori_after.show() + val cl_types_after = df_ori_after.schema.fields.map(f=>f.dataType) + val t_after = cl_types_after.mkString(":") + logInfo(s"***dsv2-flow-test*** Column type after is $t_after") + + assert(df_ori_after.count == df_ori_before.count) + assert(cl_types_after.deep == cl_types_orignal.deep) + } + + import org.apache.spark.sql.functions._ test("JDBCV2 write overwrite test") { - // Overwrite a existing table with a new schema and values. + // Overwrite(w/o truncate) a existing table with a new schema and values. val df1 = create_test_df() - // Overwrite test. Overwrite mode create a new table if it does not exist - df1.write.format("jdbcv2").mode("overwrite").option("url",jdbcUrl).option("dbtable","dsv2testTbl").save() - val df2 = spark.read.format("jdbc").option("url",jdbcUrl).option("dbtable","dsv2testTbl").load() + df1.filter(col("j") > lit(1)).write + .format("jdbcv2") + .mode("overwrite") + .option("url",jdbcUrl) + .option("truncate","false") + .option("dbtable","dsv2testTbl") + .save() + val df2 = spark.read + .format("jdbc") + .option("url",jdbcUrl) + .option("dbtable","dsv2testTbl") + .load() df2.show() + assert(df1.count -1 == df2.count) } test("Basic test") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala index 0f5e03f5258b..bd7dc7911eac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTable.scala @@ -51,8 +51,8 @@ case class DBTable (sparkSession: SparkSession, * 1. Will this be called for every 'append' request to get schema from DBTable. * If so, schema check in append should not be required. * 2. This is called in overwrite as well. What if the table does not exist. A empty - * schema would be returned resulting in an exception. Should Overwrite semantics - * not create a table. + * schema would be returned resulting in an exception. Overwrite semantics + * do not create a table. So returning a Nil would result in an error. */ logInfo("***dsv2-flows*** schema called") val schemaInDB = JdbcUtils.getSchemaOption(conn, userOptions) @@ -75,5 +75,5 @@ case class DBTable (sparkSession: SparkSession, } object DBTable { - private val CAPABILITIES = Set(BATCH_READ, BATCH_WRITE, TRUNCATE, OVERWRITE_BY_FILTER).asJava + private val CAPABILITIES = Set(BATCH_READ, BATCH_WRITE, TRUNCATE).asJava } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala index 99fc8539d982..fc1c79075f26 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/DBTableScan.scala @@ -32,7 +32,7 @@ class DBTableScan(options: JDBCOptions, def readSchema: StructType = { logInfo("***dsv2-flows*** readSchema called") - table_schema.getOrElse(StructType(Nil)) + prunedCols } override def toBatch() : Batch = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala index 2dd141df70d7..bb828995d7e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala @@ -75,17 +75,19 @@ class JDBCWriteBuilder(options: JdbcOptionsInWrite, this } - def processOverwrite() : Boolean = { - /* Overwrite table logic - 1. Check if table exists. If not create it here. Should create be done?? - 2. If table exists and isTruncate, then just truncate existing table - 3. If table exists and !isTruncate, then recreate table with new schema - Post table creation, send requests to executors to insert data. - - check filters. - */ + def processOverwrite() : Unit = { logInfo("***dsv2-flows*** processOverwrite called") - false + isTruncate match { + case true => + logInfo("***dsv2-flows*** truncating Table") + JdbcUtils.truncateTable(conn, options) + + case false => + logInfo("***dsv2-flows*** Dropping Table") + // JdbcUtils.dropTable(conn, options.table, options) + logInfo("***dsv2-flows*** Recreating table with passed schema") + Utils.createTable(conn, fwPassedSchema) + } } def processAppend() : Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md index 64e8d981baa9..dbaca7ff67dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md @@ -1,34 +1,71 @@ # Plan/Status of ongoing work on DataSource V2 JDBC connector ## Plan -| Work Item | Who's on it | Status | -|-----------------------------------------------| ----------- | ------ | -| Batch write ( append, overwrite, truncate) | shivsood | WIP | -| Streaming write | TBD | | -| Read path implementation | shivsood | WIP | -| Streaming read | TBD | | -| ?? | TBD | | +| Work Item | Who's on it | Status | Notes | +|-----------------------------------------------| ----------- | ------ | ----- | +| Batch write ( append) | shivsood | WIP | 1,6 | +| Batch write ( overwrite w truncate) | shivsood | WIP | | +| Batch write ( overwrite w/o truncate) | shivsood | WIP | 2,3 | +| Read implementation w/o column pruning | shivsood | Done | | +| Read w/o column pruning and filters | shivsood | ISSUES | 4 | +| Columnar read | TBD | | | +| Streaming write | TBD | | | +| Streaming read | TBD | | | +| Transactional write | shivsood | ISSUES | 5 | -Status -> WIP ( Work in Progress), ReadyForReview, Done + +Status -> +WIP ( Work in Progress), +DONE ( implementation is done and tested), +ISSUES (blocking issues) ## Others - Working branch is https://github.com/shivsood/spark-dsv2 - Intrested in contribution? Add work item and your name against it and party on. -## Issues/Questions/Mentions -- mode(overwrite) - what's the overwrite semantics in V2? +## Notes +1. mode(append) - what's the append semantics in V2. + V1 append would throw an exception if the df col types are not same as table schema. Is that in v2 + achieved by Table::schema? schema returns source schema (as in db) and framework checks if the source schema is + compatible with the dataframe type. Tested diff is number of cols and framework would raise an exception , + but diff in data type did not raise any exception. +2. mode(overwrite) - what's the overwrite semantics in V2? V1 overwrite will create a table if that does not exist. In V2 it seems that overwrite does not support create semantics. overwrite with a non-existing table failed following tables:schema() which returned null schema indicating no existing table. If not create scematics, then what's the diffrence between append and overwrite without column filters. -- mode(overwrite) - why overwrite results in call to truncate(). Was expecting call to overwrite() instead. -- mode(append) - what's the append semantics in V2. - V1 append would throw an exception if the df col types are not same as table schema. Is that in v2 - achieved by Table::schema? schema returns source schema (as in db) and framework checks if the source schema is - compatible with the dataframe type. Tested diffirence is number of cols and framework would raise an exception , - but diffrence in data type did not raise any exception. +3. mode(overwrite) - why overwrite results in call to WriteBuilder::truncate(). Was expecting call to + overwrite() instead. Same issues even if option("truncate","false") is explicitly specified during df.write. + OverwriteByExpressionExec::execute() in WriteToDatasourceV2Exec.scala does overwrite only when + filters are present. No clear why. +4. Read with column pruning fails with makeFromDriverError + //df2.select("i").show(10) after df.read.format("jdbc2") c.f test("JDBCV2 read test") + in MsSqlServerIntegrationSuite + Error seen as below + 19/07/31 15:20:55 INFO DBPartitionReader: ***dsv2-flows*** close called. number of rows retrieved is 0 + 19/07/31 15:20:55 ERROR Executor: Exception in task 0.0 in stage 4.0 (TID 4) + com.microsoft.sqlserver.jdbc.SQLServerException: The index 2 is out of range. + at com.microsoft.sqlserver.jdbc.SQLServerException.makeFromDriverError(SQLServerException.java:228) + at com.microsoft.sqlserver.jdbc.SQLServerResultSet.verifyValidColumnIndex(SQLServerResultSet.java:570) + at com.microsoft.sqlserver.jdbc.SQLServerResultSet.getterGetColumn(SQLServerResultSet.java:2012) + at com.microsoft.sqlserver.jdbc.SQLServerResultSet.getValue(SQLServerResultSet.java:2041) + +5. Transactional write - Does not seem feasible with the current FW. The FW suggest that executor send a commit message + to Driver, and actual commit should only be done by the driver after receiving all commit confirmations. Dont see + this feasible in JDBC as commit has to happen in JDBCConnection which is maintained by the TASKs and JDBCConnection + is not serializable that it can be sent to the Driver. + A slightly better solution may be a 2-way commit. Executors send a 'ReadyToCommit' to Driver. Driver having received + all 'ReadyToCommit' messages from executors should then send "Commit" to all executors again and then executors + can commit. + +6. write flow - currently auto commits. This needs a fix ( set autocommit in JDBC connection to false and + then commit it batches) + +7. Bulk write - no provision for writing rows in bulk. Rows are provided one at a time. + + - Lots of trivial logging. Draft implementation with API understanding as main goal -Update date : 7/22 +Update date : 7/31 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala index 88103ca0ec6e..a12d30702cbe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Utils.scala @@ -44,9 +44,10 @@ object Utils extends Logging{ } } - def createTable(structType: StructType): Unit = { + def createTable(conn : Connection, structType: StructType): Unit = { /* Create table per passed schema. Raise exception on any failure. */ + logInfo(s"***dsv2-flows*** Create Table is not implemented as yet!!" ) } def strictSchemaCheck(schemaInSpark: StructType, dbTableSchema: StructType) : Boolean = { From d1e3142f92a82a1ddb2f0df4e15af9072df5199c Mon Sep 17 00:00:00 2001 From: shivsood Date: Fri, 2 Aug 2019 10:59:24 -0700 Subject: [PATCH 23/24] Implementation for Overwrite(truncate=false). Sematics is DROP TABLE and CREATE TABLE. Reuses JDBCUtils to DROP and CREATE. JDBCUtils had to be refactored to take schema rather than dataframe. Functions that Dataframe are retained V1 compatibility. The V2 implementtion is not e2e tested as FW continues to send truncate rather than overwrite. V1 Regression test following JDBCUtils change UnitTest (./build/mvn -pl :spark-sql_2.12 clean install) were run. Test passed with regular failures that are see on master branch also. Total number of tests run: 5896 Suites: completed 288, aborted 0 Tests: succeeded 5893, failed 3, canceled 1, ignored 45, pending 0 V1 Integration Test (./build/mvn test -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.12) were run and all passed Run completed in 36 seconds, 352 milliseconds. Total number of tests run: 22 Suites: completed 5, aborted 1 Tests: succeeded 22, failed 0, canceled 0, ignored 6, pending 0 --- .../datasources/jdbc/JdbcUtils.scala | 58 +++++++++++++++---- .../v2/jdbc/JDBCWriteBuilder.scala | 4 +- .../execution/datasources/v2/jdbc/Readme.md | 14 ++--- 3 files changed, 55 insertions(+), 21 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index 63f62b31f89e..7ecb8a8605ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -729,18 +729,28 @@ object JdbcUtils extends Logging { } /** - * Compute the schema string for this RDD. + * Compute the schema string for this RDD given schema as StructType */ def schemaString( df: DataFrame, url: String, createTableColumnTypes: Option[String] = None): String = { + val userSpecifiedColTypesMap = createTableColumnTypes + .map(parseUserSpecifiedCreateTableColumnTypes( + df.sparkSession.sessionState.conf.resolver, + df.sparkSession.sessionState.conf.caseSensitiveAnalysis, + df.schema, _)).getOrElse(Map.empty[String, String]) + + schemaString(df.schema, url, userSpecifiedColTypesMap) + } + + def schemaString( + cols: StructType, + url: String, + userSpecifiedColTypesMap : Map[String, String]): String = { val sb = new StringBuilder() val dialect = JdbcDialects.get(url) - val userSpecifiedColTypesMap = createTableColumnTypes - .map(parseUserSpecifiedCreateTableColumnTypes(df, _)) - .getOrElse(Map.empty[String, String]) - df.schema.fields.foreach { field => + cols.fields.foreach { field => val name = dialect.quoteIdentifier(field.name) val typ = userSpecifiedColTypesMap .getOrElse(field.name, getJdbcType(field.dataType, dialect).databaseTypeDefinition) @@ -756,7 +766,9 @@ object JdbcUtils extends Logging { * use in-place of the default data type. */ private def parseUserSpecifiedCreateTableColumnTypes( - df: DataFrame, + resolver : Resolver, + caseType : Boolean, + cols : StructType, createTableColumnTypes: String): Map[String, String] = { def typeName(f: StructField): String = { // char/varchar gets translated to string type. Real data type specified by the user @@ -769,24 +781,22 @@ object JdbcUtils extends Logging { } val userSchema = CatalystSqlParser.parseTableSchema(createTableColumnTypes) - val nameEquality = df.sparkSession.sessionState.conf.resolver // checks duplicate columns in the user specified column types. SchemaUtils.checkColumnNameDuplication( - userSchema.map(_.name), "in the createTableColumnTypes option value", nameEquality) + userSchema.map(_.name), "in the createTableColumnTypes option value", resolver) // checks if user specified column names exist in the DataFrame schema userSchema.fieldNames.foreach { col => - df.schema.find(f => nameEquality(f.name, col)).getOrElse { + cols.find(f => resolver(f.name, col)).getOrElse { throw new AnalysisException( s"createTableColumnTypes option column $col not found in schema " + - df.schema.catalogString) + cols.catalogString) } } val userSchemaMap = userSchema.fields.map(f => f.name -> typeName(f)).toMap - val isCaseSensitive = df.sparkSession.sessionState.conf.caseSensitiveAnalysis - if (isCaseSensitive) userSchemaMap else CaseInsensitiveMap(userSchemaMap) + if (caseType) userSchemaMap else CaseInsensitiveMap(userSchemaMap) } /** @@ -855,6 +865,30 @@ object JdbcUtils extends Logging { options: JdbcOptionsInWrite): Unit = { val strSchema = schemaString( df, options.url, options.createTableColumnTypes) + + createTableWithSchemaString(conn, options, strSchema) + } + + def createTable( + conn: Connection, + schema : StructType, + options: JdbcOptionsInWrite): Unit = { + + val userSpecifiedColTypesMap = options.createTableColumnTypes + .map(parseUserSpecifiedCreateTableColumnTypes( + org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution, + true, + schema, _)).getOrElse(Map.empty[String, String]) + val strSchema = schemaString( + schema, options.url, userSpecifiedColTypesMap) + + createTableWithSchemaString(conn, options, strSchema) + } + + def createTableWithSchemaString( + conn : Connection, + options : JdbcOptionsInWrite, + strSchema : String) : Unit = { val table = options.table val createTableOptions = options.createTableOptions // Create the table if the table does not exist. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala index bb828995d7e2..591c00cb152e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCWriteBuilder.scala @@ -84,9 +84,9 @@ class JDBCWriteBuilder(options: JdbcOptionsInWrite, case false => logInfo("***dsv2-flows*** Dropping Table") - // JdbcUtils.dropTable(conn, options.table, options) + JdbcUtils.dropTable(conn, options.table, options) logInfo("***dsv2-flows*** Recreating table with passed schema") - Utils.createTable(conn, fwPassedSchema) + JdbcUtils.createTable(conn, fwPassedSchema, options) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md index dbaca7ff67dc..d543a8034144 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md @@ -3,25 +3,25 @@ ## Plan | Work Item | Who's on it | Status | Notes | |-----------------------------------------------| ----------- | ------ | ----- | -| Batch write ( append) | shivsood | WIP | 1,6 | -| Batch write ( overwrite w truncate) | shivsood | WIP | | -| Batch write ( overwrite w/o truncate) | shivsood | WIP | 2,3 | +| Batch write ( append) | shivsood | Done | 1,6 | +| Batch write ( overwrite w truncate) | shivsood | Done | | +| Batch write ( overwrite w/o truncate) | shivsood | Issues | 2,3 | | Read implementation w/o column pruning | shivsood | Done | | -| Read w/o column pruning and filters | shivsood | ISSUES | 4 | +| Read w/o column pruning and filters | shivsood | Issues | 4 | | Columnar read | TBD | | | | Streaming write | TBD | | | | Streaming read | TBD | | | -| Transactional write | shivsood | ISSUES | 5 | +| Transactional write | shivsood | Issues | 5 | Status -> WIP ( Work in Progress), DONE ( implementation is done and tested), -ISSUES (blocking issues) +Issues (blocking issues) ## Others - Working branch is https://github.com/shivsood/spark-dsv2 -- Intrested in contribution? Add work item and your name against it and party on. +- Interested in contribution? Add work item and your name against it and party on. ## Notes 1. mode(append) - what's the append semantics in V2. From 38b80e20b46135bf8ce6f227022736e2799fa2fc Mon Sep 17 00:00:00 2001 From: shivsood Date: Fri, 2 Aug 2019 12:58:57 -0700 Subject: [PATCH 24/24] minor readme updates --- .../sql/execution/datasources/v2/jdbc/Readme.md | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md index d543a8034144..25d28896e26e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/Readme.md @@ -8,16 +8,19 @@ | Batch write ( overwrite w/o truncate) | shivsood | Issues | 2,3 | | Read implementation w/o column pruning | shivsood | Done | | | Read w/o column pruning and filters | shivsood | Issues | 4 | +| Transactional write | shivsood | Blocked | 5 | +| Batch write with Filters | TBD | | | | Columnar read | TBD | | | | Streaming write | TBD | | | | Streaming read | TBD | | | -| Transactional write | shivsood | Issues | 5 | + Status -> WIP ( Work in Progress), DONE ( implementation is done and tested), -Issues (blocking issues) +Issues (Problems that need resolution) +Blocked ( B ## Others - Working branch is https://github.com/shivsood/spark-dsv2 @@ -42,6 +45,8 @@ Issues (blocking issues) //df2.select("i").show(10) after df.read.format("jdbc2") c.f test("JDBCV2 read test") in MsSqlServerIntegrationSuite Error seen as below + + `` 19/07/31 15:20:55 INFO DBPartitionReader: ***dsv2-flows*** close called. number of rows retrieved is 0 19/07/31 15:20:55 ERROR Executor: Exception in task 0.0 in stage 4.0 (TID 4) com.microsoft.sqlserver.jdbc.SQLServerException: The index 2 is out of range. @@ -49,7 +54,8 @@ Issues (blocking issues) at com.microsoft.sqlserver.jdbc.SQLServerResultSet.verifyValidColumnIndex(SQLServerResultSet.java:570) at com.microsoft.sqlserver.jdbc.SQLServerResultSet.getterGetColumn(SQLServerResultSet.java:2012) at com.microsoft.sqlserver.jdbc.SQLServerResultSet.getValue(SQLServerResultSet.java:2041) - + `` + 5. Transactional write - Does not seem feasible with the current FW. The FW suggest that executor send a commit message to Driver, and actual commit should only be done by the driver after receiving all commit confirmations. Dont see this feasible in JDBC as commit has to happen in JDBCConnection which is maintained by the TASKs and JDBCConnection