You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by tribbloid <gi...@git.apache.org> on 2015/09/17 00:38:22 UTC

[GitHub] spark pull request: Spark 10625

GitHub user tribbloid opened a pull request:

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

    Spark 10625

    Connection properties are now deep copied before they are used by JDBC Drivers, this solvs all problems in unit tests

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

    $ git pull https://github.com/Schedule1/spark SPARK-10625

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

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

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

    This closes #8785
    
----
commit 7d6d6b96c23a74b66212acb617113d11435890be
Author: tribbloid <pc...@uow.edu.au>
Date:   2015-09-15T23:02:57Z

    test case demonstrating SPARK-10625:
    Spark SQL JDBC read/write is unable to handle JDBC Drivers that adds unserializable objects into connection properties

commit d807c9b76ff0364eb49a9e9af0b92437e13b3811
Author: tribbloid <pc...@uow.edu.au>
Date:   2015-09-16T22:36:05Z

    add one more unit test
    fix JDBCRelation & DataFrameWriter to pass all tests

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #8785: [Spark-10625] [SQL] Spark SQL JDBC read/write is unable t...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/8785
  
    I am not supposed to decide what to merge but I left the command as I just found this seems not active to the review comments and I assumed that this PR is currently abandoned which the author happened to be not able to proceed further for now.
    
    I'd rebase/address the review comments and keep pinging the related guys here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48715367
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -108,6 +122,6 @@ private[sql] case class JDBCRelation(
       override def insert(data: DataFrame, overwrite: Boolean): Unit = {
         data.write
           .mode(if (overwrite) SaveMode.Overwrite else SaveMode.Append)
    -      .jdbc(url, table, properties)
    +      .jdbc(url, table, JDBCRelation.getEffectiveProperties(properties))
    --- End diff --
    
    This call is effectively just cloning the properties, but we're already doing the clone inside of `.jdbc()` itself, so we don't need this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48678521
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.scala ---
    @@ -0,0 +1,53 @@
    +/*
    + * 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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  import scala.collection.JavaConverters._
    --- End diff --
    
    That's a reasonable argument, though we have imported this set of implicits widely elsewhere in the code


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-167015238
  
    Of course Dr. I'll finish reading in a few hours.
    
    Yours Peng
    
    On 15-12-23 05:10 AM, Sean Owen wrote:
    >
    > @tribbloid <https://github.com/tribbloid> let's wrap this up at last. 
    > Can you please review all the outstanding comments and address them or 
    > else close this?
    >
    > —
    > Reply to this email directly or view it on GitHub 
    > <https://github.com/apache/spark/pull/8785#issuecomment-166854056>.
    >
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Spark 10625

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r39704595
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala ---
    @@ -450,4 +451,34 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext
         assert(db2Dialect.getJDBCType(StringType).map(_.databaseTypeDefinition).get == "CLOB")
         assert(db2Dialect.getJDBCType(BooleanType).map(_.databaseTypeDefinition).get == "CHAR(1)")
       }
    +
    +  test("Basic API with Unserializable Driver Properties") {
    +    object UnserializableH2Driver extends org.h2.Driver {
    +
    +      override def connect(url: String, info: Properties): Connection = {
    +
    +        val result = super.connect(url, info)
    +        info.put("unserializableDriver", this)
    +        result
    +      }
    +
    +      override def getParentLogger: Logger = ???
    +    }
    +
    +    import scala.collection.JavaConversions._
    +
    +    val oldDrivers = DriverManager.getDrivers.filter(_.acceptsURL("jdbc:h2:")).toSeq
    --- End diff --
    
    Thanks a lot Holden, I've tried that, but apparently it takes many classes and functions to override org.h2.Driver's valid schema. So I took the shortcut.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-168493283
  
    **[Test build #2301 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2301/consoleFull)** for PR 8785 at commit [`6a240b1`](https://github.com/apache/spark/commit/6a240b12ec994ddb989a22b46b5b90a3e9fdc014).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r47730829
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  def replaceDriverDuring[T](f: => T): T = {
    +    import scala.collection.JavaConverters._
    +
    +    object UnserializableH2Driver extends org.h2.Driver {
    +
    +      override def connect(url: String, info: Properties): Connection = {
    +
    +        val result = super.connect(url, info)
    +        info.put("unserializableDriver", this)
    +        result
    +      }
    +
    +      override def getParentLogger: Logger = null
    +    }
    +
    +    val oldDrivers = DriverManager.getDrivers.asScala.filter(_.acceptsURL("jdbc:h2:")).toSeq
    +    oldDrivers.foreach{
    +      DriverManager.deregisterDriver
    +    }
    +    DriverManager.registerDriver(UnserializableH2Driver)
    +
    +    val result = try {
    +      f
    +    }
    +    finally {
    --- End diff --
    
    same as before


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-163455378
  
    @srowen Sorry you are right, there is already a deep copy and I should just use that, will correct immediately


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r51035587
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala ---
    @@ -469,6 +469,13 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext
         assert(derbyDialect.getJDBCType(BooleanType).map(_.databaseTypeDefinition).get == "BOOLEAN")
       }
     
    +  test("Spark-10625: JDBC read should allow driver to insert unserializable into properties") {
    --- End diff --
    
    fixed!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #8785: [Spark-10625] [SQL] Spark SQL JDBC read/write is unable t...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the issue:

    https://github.com/apache/spark/pull/8785
  
    All conflict fixed with minimal changes to original patch that has been peer-reviewed in Jan 2016. Request for merging.
    
    **WARNING**: DataFrameWriter Line 402 (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala#L402)
    
    can use JDBCRelation.getEffectiveProperties Line106
    (https://github.com/apache/spark/pull/8785/files#diff-5f0d0643fcfad315df0fdd7cae52dfaeR106)
    
    but I didn't change it to minimize diff. Please advise if it has to be corrected.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-168148505
  
    **[Test build #2280 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2280/consoleFull)** for PR 8785 at commit [`f6c0fde`](https://github.com/apache/spark/commit/f6c0fdeb839b43d9009d61187cb052f274ab11d2).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #8785: [Spark-10625] [SQL] Spark SQL JDBC read/write is unable t...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the issue:

    https://github.com/apache/spark/pull/8785
  
    This needs a rebase and there are still outstanding review comments (minor ones)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-162701698
  
    @srowen thanks a lot for posting the problem in import declarations, I've already correct it and won't optimize import habitually.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #8785: [Spark-10625] [SQL] Spark SQL JDBC read/write is unable t...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/8785
  
    ping @tribbloid. Are you able to proceed the review comments? If not, it'd be better closed for now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48194924
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.scala ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  def replaceDriverDuring[T](f: => T): T = {
    +    import scala.collection.JavaConverters._
    +
    +    object UnserializableH2Driver extends org.h2.Driver {
    +
    +      override def connect(url: String, info: Properties): Connection = {
    +
    +        val result = super.connect(url, info)
    +        info.put("unserializableDriver", this)
    +        result
    +      }
    +
    +      override def getParentLogger: Logger = null
    +    }
    +
    +    val oldDrivers = DriverManager.getDrivers.asScala.filter(_.acceptsURL("jdbc:h2:"))
    +    oldDrivers.foreach{ DriverManager.deregisterDriver }
    +    DriverManager.registerDriver(UnserializableH2Driver)
    +
    +    val result = try { f }
    +    finally {
    --- End diff --
    
    This still isn't normal try-finally formatting
    
    ```
    try {
    
    } finally {
    
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48460773
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.scala ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  def replaceDriverDuring[T](f: => T): T = {
    +    import scala.collection.JavaConverters._
    +
    +    object UnserializableH2Driver extends org.h2.Driver {
    +
    +      override def connect(url: String, info: Properties): Connection = {
    +
    +        val result = super.connect(url, info)
    +        info.put("unserializableDriver", this)
    +        result
    +      }
    +
    +      override def getParentLogger: Logger = null
    +    }
    +
    +    val oldDrivers = DriverManager.getDrivers.asScala.filter(_.acceptsURL("jdbc:h2:"))
    +    oldDrivers.foreach{ DriverManager.deregisterDriver }
    --- End diff --
    
    Yes, both foreach are made one line


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48636488
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.scala ---
    @@ -0,0 +1,53 @@
    +/*
    + * 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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  import scala.collection.JavaConverters._
    --- End diff --
    
    OK, I was under the false impression that import of implicits may cause conflicts. But in a well designed library this rarely happens


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r47773368
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -75,6 +75,18 @@ private[sql] object JDBCRelation {
         }
         ans.toArray
       }
    +
    +  def getEffectiveProperties(
    +      connectionProperties: Properties,
    +      extraOptions: scala.collection.Map[String, String] = Map()): Properties = {
    +    import scala.collection.JavaConverters._
    --- End diff --
    
    Here and below I don't see a need to scope this import so locally. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48194955
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.scala ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  def replaceDriverDuring[T](f: => T): T = {
    +    import scala.collection.JavaConverters._
    --- End diff --
    
    It's not necessary to make this local


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Spark-10625: Spark SQL JDBC read/write is unab...

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-141532159
  
    Looks like good improvements, less duplicated code, although there are still some minor style issues. You might want to merge in the latest master branch so that tests can be run.
    Also as a note: to help relevant reviewers find your PR faster, https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-PullRequest has some steps on how to name the PR (essentially [SPARK-ISSUENUMBER][COMPONENT] - description) in this case maybe something like starting the PR's title with [SPARK-10625][SQL] will help the PR be visible to the people most able to help with the review.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #8785: [Spark-10625] [SQL] Spark SQL JDBC read/write is unable t...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the issue:

    https://github.com/apache/spark/pull/8785
  
    @tribbloid is this a problem that needs to be fixed?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r50641262
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -37,6 +37,9 @@ private[sql] case class JDBCPartitioningInfo(
         numPartitions: Int)
     
     private[sql] object JDBCRelation {
    +
    +  import scala.collection.JavaConverters._
    --- End diff --
    
    Hey @tribbloid, #10519 has now been merged, so I think things should be in a much more testable state now in case you want to revive this PR and try to get it merged.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-174732217
  
    Good! Boots on the ground.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-167539724
  
    **[Test build #2258 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2258/consoleFull)** for PR 8785 at commit [`6af8fd8`](https://github.com/apache/spark/commit/6af8fd8b824f5f343a01868560b74a1f55acd02f).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-175203838
  
    Hey, there's still a bunch of review comments that I left which haven't been acknowledged or addressed! Mind replying to them?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-178820992
  
    Hi Josh, do you see any problem? The Jenkins should have it tested and its clean to be merged be now


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48715260
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -88,7 +101,8 @@ private[sql] case class JDBCRelation(
     
       override val needConversion: Boolean = false
     
    -  override val schema: StructType = JDBCRDD.resolveTable(url, table, properties)
    +  override val schema: StructType =
    +    JDBCRDD.resolveTable(url, table, JDBCRelation.getEffectiveProperties(properties))
    --- End diff --
    
    The non-serializability doesn't come into play here since this runs entirely on the driver, right? Or can this method wind up being called on executors?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r47206801
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -75,6 +76,19 @@ private[sql] object JDBCRelation {
         }
         ans.toArray
       }
    +
    +  def getEffectiveProperties(
    --- End diff --
    
    @tribbloid I think Holden's comment still stands -- see how other methods wrap args. I also don't think you need to fully-qualify `scala.collection.Map` here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48195057
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -75,6 +78,16 @@ private[sql] object JDBCRelation {
         }
         ans.toArray
       }
    +
    +  def getEffectiveProperties(
    +      connectionProperties: Properties,
    +      extraOptions: scala.collection.Map[String, String] = Map()): Properties = {
    --- End diff --
    
    Still need not be qualified; then I think this need not wrap?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-168493407
  
    **[Test build #2304 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2304/consoleFull)** for PR 8785 at commit [`6a240b1`](https://github.com/apache/spark/commit/6a240b12ec994ddb989a22b46b5b90a3e9fdc014).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48715790
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.scala ---
    @@ -0,0 +1,53 @@
    +/*
    + * 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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  import scala.collection.JavaConverters._
    +
    +  def replaceDriverDuring[T](f: => T): T = {
    +    object UnserializableH2Driver extends org.h2.Driver {
    +
    +      override def connect(url: String, info: Properties): Connection = {
    +
    +        val result = super.connect(url, info)
    +        info.put("unserializableDriver", this)
    +        result
    +      }
    +
    +      override def getParentLogger: Logger = null
    +    }
    +
    +    val oldDrivers = DriverManager.getDrivers.asScala.filter(_.acceptsURL("jdbc:h2:"))
    +    oldDrivers.foreach(DriverManager.deregisterDriver)
    +    DriverManager.registerDriver(UnserializableH2Driver)
    --- End diff --
    
    Should probably be inside of the `try` block, no?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-163455429
  
    done! Thanks a lot for pointing it out!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r47790709
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -75,6 +75,18 @@ private[sql] object JDBCRelation {
         }
         ans.toArray
       }
    +
    +  def getEffectiveProperties(
    +      connectionProperties: Properties,
    +      extraOptions: scala.collection.Map[String, String] = Map()): Properties = {
    +    import scala.collection.JavaConverters._
    --- End diff --
    
    already corrected by moving to object-level scope


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #8785: [Spark-10625] [SQL] Spark SQL JDBC read/write is unable t...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/8785
  
    Can one of the admins verify this patch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #8785: [Spark-10625] [SQL] Spark SQL JDBC read/write is u...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-166854056
  
    @tribbloid let's wrap this up at last. Can you please review all the outstanding comments and address them or else close this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #8785: [Spark-10625] [SQL] Spark SQL JDBC read/write is unable t...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the issue:

    https://github.com/apache/spark/pull/8785
  
    Finally someone replied :)
    Do you suggest me to fix it now? Or wait until 2.0 RC code has become a bit more stable (but not frozen)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r51038017
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -75,6 +78,16 @@ private[sql] object JDBCRelation {
         }
         ans.toArray
       }
    +
    +  def getEffectiveProperties(
    +      connectionProperties: Properties,
    +      extraOptions: scala.collection.Map[String, String] = Map()): Properties = {
    +    val props = new Properties()
    +    props.putAll(extraOptions.asJava)
    +    // connectionProperties should override settings in extraOptions
    +    props.putAll(connectionProperties)
    --- End diff --
    
    yes they can be preserved by the deep copy, but unfortunately any changes on parent properties AFTER the deep copy won't be preserved in returned value. I expect callers of this function to use its returned value transiently.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48715673
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.scala ---
    @@ -0,0 +1,53 @@
    +/*
    + * 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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  import scala.collection.JavaConverters._
    --- End diff --
    
    Please move this import to the top; the arguments RE: scope of implicits are different on a case-by-case basis, but JavaConverters is a case that should pretty much always be safe to put in a somewhat global scope.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-165577580
  
    @tribbloid I think there are still some little style comments in `UnserializableDriverHelper`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Spark-10625: Spark SQL JDBC read/write is unab...

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r39886832
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala ---
    @@ -450,4 +449,12 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext
         assert(db2Dialect.getJDBCType(StringType).map(_.databaseTypeDefinition).get == "CLOB")
         assert(db2Dialect.getJDBCType(BooleanType).map(_.databaseTypeDefinition).get == "CHAR(1)")
       }
    +
    +  test("Basic API with Unserializable Driver Properties") {
    +
    --- End diff --
    
    extra blank line


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48715834
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -37,6 +37,9 @@ private[sql] case class JDBCPartitioningInfo(
         numPartitions: Int)
     
     private[sql] object JDBCRelation {
    +
    +  import scala.collection.JavaConverters._
    --- End diff --
    
    Same here; move this to the top with the other Scala import.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-175818906
  
    Yes, all comments resolved.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-168087181
  
    I'm so done with IntelliJ autocorrection. But all fixed, thanks a lot!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48460762
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.scala ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  def replaceDriverDuring[T](f: => T): T = {
    +    import scala.collection.JavaConverters._
    --- End diff --
    
    Yes, promote to under object


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-168148300
  
    **[Test build #2280 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2280/consoleFull)** for PR 8785 at commit [`f6c0fde`](https://github.com/apache/spark/commit/f6c0fdeb839b43d9009d61187cb052f274ab11d2).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r51035933
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -88,7 +101,8 @@ private[sql] case class JDBCRelation(
     
       override val needConversion: Boolean = false
     
    -  override val schema: StructType = JDBCRDD.resolveTable(url, table, properties)
    +  override val schema: StructType =
    +    JDBCRDD.resolveTable(url, table, JDBCRelation.getEffectiveProperties(properties))
    --- End diff --
    
    Yes, you are right, I removed the getEffectiveProperties() and all tests passed. This may be attributed to changes in getConnectionFactory()


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Spark 10625

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r39702261
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala ---
    @@ -450,4 +451,34 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext
         assert(db2Dialect.getJDBCType(StringType).map(_.databaseTypeDefinition).get == "CLOB")
         assert(db2Dialect.getJDBCType(BooleanType).map(_.databaseTypeDefinition).get == "CHAR(1)")
       }
    +
    +  test("Basic API with Unserializable Driver Properties") {
    +    object UnserializableH2Driver extends org.h2.Driver {
    +
    +      override def connect(url: String, info: Properties): Connection = {
    +
    +        val result = super.connect(url, info)
    +        info.put("unserializableDriver", this)
    +        result
    +      }
    +
    +      override def getParentLogger: Logger = ???
    +    }
    +
    +    import scala.collection.JavaConversions._
    +
    +    val oldDrivers = DriverManager.getDrivers.filter(_.acceptsURL("jdbc:h2:")).toSeq
    --- End diff --
    
    Rather than de-registering and re-registering, would it maybe make sense to instead accept a different URL schema to skip this step?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-162705458
  
    I don't get that -- what does the original object matter if it's copied here? and how would the copy change? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #8785: [Spark-10625] [SQL] Spark SQL JDBC read/write is unable t...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the issue:

    https://github.com/apache/spark/pull/8785
  
    Please wait for me to address the conflicts, will do this after 2.0.0 preview main component become stable enough


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-165147353
  
    Good, thanks a lot for advices.
    This should be the last patch


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Spark 10625

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-140935511
  
    Hey, do you mind giving this PR a descriptive title? Makes the PR queue easier to scan.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-166392614
  
    Sorry I misunderstood your intention on previous comments, now both problem should be fixed: toSeq is removed and single line of code in brackets are pulled up.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48492460
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.scala ---
    @@ -0,0 +1,53 @@
    +/*
    + * 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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  import scala.collection.JavaConverters._
    --- End diff --
    
    I meant why is it imported inside the object scope rather than with other imports -- the use of the converters is fine and important. There are some special cases where qualified name or local imports are needed but I think by convention, the strong default is to declare all of them at the top of the compilation unit, especially fairly common ones.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-165099839
  
    I think this is looking good otherwise.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #8785: [Spark-10625] [SQL] Spark SQL JDBC read/write is unable t...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the issue:

    https://github.com/apache/spark/pull/8785
  
    yeah, just need to rebase on 2.2-SNAPSHOT+
    are you going to merge immediately after rebase + syntax validation?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-167978930
  
    @tribbloid still having some issue with the style checker:
    ```
    [error] /home/jenkins/workspace/NewSparkPullRequestBuilder/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala:42:0: Whitespace at end of line
    ```
    It's minor but does need to pass


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48460878
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.scala ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  def replaceDriverDuring[T](f: => T): T = {
    +    import scala.collection.JavaConverters._
    +
    +    object UnserializableH2Driver extends org.h2.Driver {
    +
    +      override def connect(url: String, info: Properties): Connection = {
    +
    +        val result = super.connect(url, info)
    +        info.put("unserializableDriver", this)
    +        result
    +      }
    +
    +      override def getParentLogger: Logger = null
    +    }
    +
    +    val oldDrivers = DriverManager.getDrivers.asScala.filter(_.acceptsURL("jdbc:h2:"))
    +    oldDrivers.foreach{ DriverManager.deregisterDriver }
    +    DriverManager.registerDriver(UnserializableH2Driver)
    +
    +    val result = try { f }
    +    finally {
    --- End diff --
    
    Yes, fixing try-finally formatting


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48715166
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -75,6 +76,19 @@ private[sql] object JDBCRelation {
         }
         ans.toArray
       }
    +
    +  def getEffectiveProperties(
    --- End diff --
    
    +1 on Sean's comments. Also, could you add a one- or two-line comment to explain what's going on here? Maybe give this method Scaladoc?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48715580
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -75,6 +78,16 @@ private[sql] object JDBCRelation {
         }
         ans.toArray
       }
    +
    +  def getEffectiveProperties(
    +      connectionProperties: Properties,
    +      extraOptions: scala.collection.Map[String, String] = Map()): Properties = {
    +    val props = new Properties()
    +    props.putAll(extraOptions.asJava)
    +    // connectionProperties should override settings in extraOptions
    +    props.putAll(connectionProperties)
    --- End diff --
    
    How does this handle inherited properties? E.g. if `connectionProperties` was defined by inheriting defaults from a parent set of properties, would those inherited defaults be preserved here or would they be lost? This concern existed in the old code as well, but just thought I'd ask since we're using this method in a few places in order to do defensive copying of property objects.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r47730716
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -88,7 +100,11 @@ private[sql] case class JDBCRelation(
     
       override val needConversion: Boolean = false
     
    -  override val schema: StructType = JDBCRDD.resolveTable(url, table, properties)
    +  override val schema: StructType = JDBCRDD.resolveTable(
    +    url,
    --- End diff --
    
    Yes it does, changed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r47773335
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  def replaceDriverDuring[T](f: => T): T = {
    +    import scala.collection.JavaConverters._
    +
    +    object UnserializableH2Driver extends org.h2.Driver {
    +
    +      override def connect(url: String, info: Properties): Connection = {
    +
    +        val result = super.connect(url, info)
    +        info.put("unserializableDriver", this)
    +        result
    +      }
    +
    +      override def getParentLogger: Logger = null
    +    }
    +
    +    val oldDrivers = DriverManager.getDrivers.asScala.filter(_.acceptsURL("jdbc:h2:")).toSeq
    +    oldDrivers.foreach{
    --- End diff --
    
    `asScala` should already make a `Seq`. You'll also probably want to just write `oldDrivers.foreach(DriverManager.deregisterDriver)` as I don't know if this will pass style checks (?) I think there's still the try-finally issue below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
GitHub user tribbloid reopened a pull request:

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

    [Spark-10625] [SQL] Spark SQL JDBC read/write is unable to handle JDBC Drivers that adds unserializable objects into connection properties

    Connection properties are now deep copied before they are used by JDBC Drivers, this solvs all problems in unit tests

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

    $ git pull https://github.com/Schedule1/spark SPARK-10625

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

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

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

    This closes #8785
    
----
commit 267afca7caa3e06a0fdac123f8a50033a726c564
Author: tribbloid <pc...@uow.edu.au>
Date:   2015-09-15T23:02:57Z

    test case demonstrating SPARK-10625:
    Spark SQL JDBC read/write is unable to handle JDBC Drivers that adds unserializable objects into connection properties
    
    add one more unit test
    fix JDBCRelation & DataFrameWriter to pass all tests
    
    revise scala style
    
    put driver replacement code into a shared function
    
    fix styling
    
    upgrade to master and resolve all related issues

commit cd641c01260eeabe910d819fffd9193461b37946
Author: tribbloid <pc...@uow.edu.au>
Date:   2015-12-10T01:15:18Z

    remove the useless second deep copy of properties
    rename test names to be more explicit

commit d33dc9c773e01695b3098f5a69733eacc2a88456
Author: tribbloid <pc...@uow.edu.au>
Date:   2015-12-16T02:39:37Z

    minor refactoring based on Sean's suggestion

commit 1dcede3d98575152dda1c46b263fc4e2361f0766
Author: tribbloid <pc...@uow.edu.au>
Date:   2015-12-16T15:37:04Z

    move JavaConverters import to under object

commit d2723c8a59301419af6132e06cacbb2860968c71
Author: tribbloid <pc...@uow.edu.au>
Date:   2015-12-21T19:07:05Z

    remove the redundant toSeq and pull up lines in brackets

commit 6af8fd8b824f5f343a01868560b74a1f55acd02f
Author: tribbloid <pc...@uow.edu.au>
Date:   2015-12-28T02:05:00Z

    improve styling in UnserializableDriverHelper and JDBCRelation

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r51036461
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -37,6 +37,9 @@ private[sql] case class JDBCPartitioningInfo(
         numPartitions: Int)
     
     private[sql] object JDBCRelation {
    +
    +  import scala.collection.JavaConverters._
    --- End diff --
    
     import scala.collection.JavaConverters._ moved to the top with other non-wildcard imports


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-168211061
  
    Looks like the removed qualifier of Map in JDBCRelation has caused build to fail (Map is cast to immutable.Map). I'm adding back the qualifier scala.collection.Map now


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Spark-10625: Spark SQL JDBC read/write is unab...

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r39886795
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -75,6 +76,19 @@ private[sql] object JDBCRelation {
         }
         ans.toArray
       }
    +
    +  def getEffectiveProperties(
    --- End diff --
    
    this indentation is still a little funky, see https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-142306277
  
    PR fixed as well, thanks a lot Holden!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-168622897
  
    I think this PR would be much easier to test after #10519 is fixed, in case any reviewers here would also like to look at my PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r47730696
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -75,6 +75,18 @@ private[sql] object JDBCRelation {
         }
         ans.toArray
       }
    +
    +  def getEffectiveProperties(
    +      connectionProperties: Properties,
    +      extraOptions: scala.collection.Map[String, String] = Map()): Properties = {
    +    val props = new Properties()
    +    extraOptions.foreach { case (key, value) =>
    --- End diff --
    
    Totally agree, changed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-168621202
  
    Hm @tribbloid this now fails a test on the JDBC drivers:
    
    ```
    [info] Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.jdbc.JDBCSuite *** ABORTED *** (84 milliseconds)
    [info]   java.sql.SQLException: No suitable driver found for jdbc:h2:mem:testdb0
    [info]   at java.sql.DriverManager.getConnection(DriverManager.java:596)
    [info]   at java.sql.DriverManager.getConnection(DriverManager.java:187)
    [info]   at org.apache.spark.sql.jdbc.JDBCSuite$$anonfun$43.apply(JDBCSuite.scala:62)
    [info]   at org.apache.spark.sql.jdbc.JDBCSuite$$anonfun$43.apply(JDBCSuite.scala:53)
    [info]   at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:195)
    [info]   at org.apache.spark.sql.jdbc.JDBCSuite.runTest(JDBCSuite.scala:36)
    [info]   at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
    [info]   at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
    [info]   at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413)
    [info]   at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401)
    [info]   at scala.collection.immutable.List.foreach(List.scala:318)
    [info]   at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
    [info]   at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396)
    [info]   at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483)
    [info]   at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208)
    [info]   at org.scalatest.FunSuite.runTests(FunSuite.scala:1555)
    [info]   at org.scalatest.Suite$class.run(Suite.scala:1424)
    [info]   at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555)
    [info]   at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
    [info]   at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
    [info]   at org.scalatest.SuperEngine.runImpl(Engine.scala:545)
    [info]   at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212)
    [info]   at org.apache.spark.sql.jdbc.JDBCSuite.org$scalatest$BeforeAndAfter$$super$run(JDBCSuite.scala:36)
    [info]   at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241)
    [info]   at org.apache.spark.sql.jdbc.JDBCSuite.org$scalatest$BeforeAndAfterAll$$super$run(JDBCSuite.scala:36)
    [info]   at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257)
    [info]   at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256)
    [info]   at org.apache.spark.sql.jdbc.JDBCSuite.run(JDBCSuite.scala:36)
    [info]   at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462)
    [info]   at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671)
    [info]   at sbt.ForkMain$Run$2.call(ForkMain.java:296)
    [info]   at sbt.ForkMain$Run$2.call(ForkMain.java:286)
    [info]   at java.util.concurrent.FutureTask.run(FutureTask.java:262)
    [info]   at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
    [info]   at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
    [info]   at java.lang.Thread.run(Thread.java:745)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r47206961
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -88,7 +100,11 @@ private[sql] case class JDBCRelation(
     
       override val needConversion: Boolean = false
     
    -  override val schema: StructType = JDBCRDD.resolveTable(url, table, properties)
    +  override val schema: StructType = JDBCRDD.resolveTable(
    +    url,
    --- End diff --
    
    Same, I think we may need to wrap this differently. Does it go on one line if `JDBCRDD` starts a new line?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #8785: [Spark-10625] [SQL] Spark SQL JDBC read/write is u...

Posted by clockfly <gi...@git.apache.org>.
Github user clockfly commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r66919638
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -27,6 +27,8 @@ import org.apache.spark.sql.{DataFrame, Row, SaveMode, SQLContext}
     import org.apache.spark.sql.sources._
     import org.apache.spark.sql.types.StructType
     
    +import scala.collection.JavaConverters._
    --- End diff --
    
    scala import should be grouped together with line 22.
    https://github.com/apache/spark/pull/8785/files#diff-5f0d0643fcfad315df0fdd7cae52dfaeR22


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #8785: [Spark-10625] [SQL] Spark SQL JDBC read/write is unable t...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the issue:

    https://github.com/apache/spark/pull/8785
  
    Absolutely lady, & welcome to Toronto.
    
    On 2016-07-06 04:56 PM, Holden Karau wrote:
    >
    > So 2.0.0-preview is already out and we are in RC2 so I wouldn't expect 
    > any big changes happening right now if you want to take the time to 
    > update the PR :)
    >
    > —
    > You are receiving this because you were mentioned.
    > Reply to this email directly, view it on GitHub 
    > <https://github.com/apache/spark/pull/8785#issuecomment-230903601>, or 
    > mute the thread 
    > <https://github.com/notifications/unsubscribe/ADSDBUFfM3_zycYhk0ryFSRPLrNJWhqIks5qTBZrgaJpZM4F-xmq>.
    >
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48194995
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.scala ---
    @@ -0,0 +1,54 @@
    +/*
    + * 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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  def replaceDriverDuring[T](f: => T): T = {
    +    import scala.collection.JavaConverters._
    +
    +    object UnserializableH2Driver extends org.h2.Driver {
    +
    +      override def connect(url: String, info: Properties): Connection = {
    +
    +        val result = super.connect(url, info)
    +        info.put("unserializableDriver", this)
    +        result
    +      }
    +
    +      override def getParentLogger: Logger = null
    +    }
    +
    +    val oldDrivers = DriverManager.getDrivers.asScala.filter(_.acceptsURL("jdbc:h2:"))
    +    oldDrivers.foreach{ DriverManager.deregisterDriver }
    --- End diff --
    
    Still not right w.r.t spaces: `foo.foreach(bar)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Spark 10625

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-140914199
  
    Can one of the admins verify this patch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48460893
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -75,6 +78,16 @@ private[sql] object JDBCRelation {
         }
         ans.toArray
       }
    +
    +  def getEffectiveProperties(
    +      connectionProperties: Properties,
    +      extraOptions: scala.collection.Map[String, String] = Map()): Properties = {
    --- End diff --
    
    Qualifier removed, but if without wrap the line exceeds 100 characters.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r47206871
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -75,6 +75,18 @@ private[sql] object JDBCRelation {
         }
         ans.toArray
       }
    +
    +  def getEffectiveProperties(
    +      connectionProperties: Properties,
    +      extraOptions: scala.collection.Map[String, String] = Map()): Properties = {
    +    val props = new Properties()
    +    extraOptions.foreach { case (key, value) =>
    --- End diff --
    
    Sorry, only now really thinking through this -- if you import `scala.collection.JavaConverters._` then this clause can just be `props.putAll(extraOptions.asJava)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-162213600
  
    @tribbloid are you still working on this? I had an outstanding question or two here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Spark-10625: Spark SQL JDBC read/write is unab...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-141114437
  
    Does it look any better now?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r47730785
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  def replaceDriverDuring[T](f: => T): T = {
    +    import scala.collection.JavaConverters._
    +
    +    object UnserializableH2Driver extends org.h2.Driver {
    +
    +      override def connect(url: String, info: Properties): Connection = {
    +
    +        val result = super.connect(url, info)
    +        info.put("unserializableDriver", this)
    +        result
    +      }
    +
    +      override def getParentLogger: Logger = null
    +    }
    +
    +    val oldDrivers = DriverManager.getDrivers.asScala.filter(_.acceptsURL("jdbc:h2:")).toSeq
    +    oldDrivers.foreach{
    --- End diff --
    
    IMHO not if the previous line has ```toSeq```, should I refactor to make it obvious


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Spark 10625

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r39702311
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala ---
    @@ -151,4 +152,36 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter {
         assert(2 === sqlContext.read.jdbc(url1, "TEST.PEOPLE1", properties).count)
         assert(2 === sqlContext.read.jdbc(url1, "TEST.PEOPLE1", properties).collect()(0).length)
       }
    +
    +  test("INSERT to JDBC Datasource with Unserializable Driver Properties") {
    +
    +    object UnserializableH2Driver extends org.h2.Driver {
    +
    +      override def connect(url: String, info: Properties): Connection = {
    +
    +        val result = super.connect(url, info)
    +        info.put("unserializableDriver", this)
    +        result
    +      }
    +
    +      override def getParentLogger: Logger = ???
    +    }
    +
    +    import scala.collection.JavaConversions._
    +
    +    val oldDrivers = DriverManager.getDrivers.filter(_.acceptsURL("jdbc:h2:")).toSeq
    --- End diff --
    
    Same comment as before (although if it doesn't make sense to change the accept urls maybe we could put this in a shared function).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r47207095
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  def replaceDriverDuring[T](f: => T): T = {
    +    import scala.collection.JavaConverters._
    +
    +    object UnserializableH2Driver extends org.h2.Driver {
    +
    +      override def connect(url: String, info: Properties): Connection = {
    +
    +        val result = super.connect(url, info)
    +        info.put("unserializableDriver", this)
    +        result
    +      }
    +
    +      override def getParentLogger: Logger = null
    +    }
    +
    +    val oldDrivers = DriverManager.getDrivers.asScala.filter(_.acceptsURL("jdbc:h2:")).toSeq
    +    oldDrivers.foreach{
    +      DriverManager.deregisterDriver
    +    }
    +    DriverManager.registerDriver(UnserializableH2Driver)
    +
    +    val result = try {
    +      f
    +    }
    +    finally {
    --- End diff --
    
    Pull this up to the previous line. Same comment about foreach below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48714853
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala ---
    @@ -57,4 +57,3 @@ object DriverRegistry extends Logging {
         case driver => driver.getClass.getCanonicalName
       }
     }
    -
    --- End diff --
    
    Can you please minimize unrelated whitespace changes?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r47207033
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  def replaceDriverDuring[T](f: => T): T = {
    +    import scala.collection.JavaConverters._
    +
    +    object UnserializableH2Driver extends org.h2.Driver {
    +
    +      override def connect(url: String, info: Properties): Connection = {
    +
    +        val result = super.connect(url, info)
    +        info.put("unserializableDriver", this)
    +        result
    +      }
    +
    +      override def getParentLogger: Logger = null
    +    }
    +
    +    val oldDrivers = DriverManager.getDrivers.asScala.filter(_.acceptsURL("jdbc:h2:")).toSeq
    +    oldDrivers.foreach{
    --- End diff --
    
    `oldDrivers.foreach(DriverManager.deregisterDriver)`. Do you need `toSeq` above?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-164968520
  
    Thanks a lot! All addressed. Sorry I was distracted from this ticket for 2 days


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48715771
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.scala ---
    @@ -0,0 +1,53 @@
    +/*
    + * 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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  import scala.collection.JavaConverters._
    +
    +  def replaceDriverDuring[T](f: => T): T = {
    +    object UnserializableH2Driver extends org.h2.Driver {
    +
    +      override def connect(url: String, info: Properties): Connection = {
    +
    +        val result = super.connect(url, info)
    +        info.put("unserializableDriver", this)
    +        result
    +      }
    +
    +      override def getParentLogger: Logger = null
    +    }
    +
    +    val oldDrivers = DriverManager.getDrivers.asScala.filter(_.acceptsURL("jdbc:h2:"))
    --- End diff --
    
    I'd add an explicit `.toSeq` at the end; what's probably happening is that this is becoming a one-shot scala iterator, which is causing the finally block to not re-register the old drivers.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #8785: [Spark-10625] [SQL] Spark SQL JDBC read/write is unable t...

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on the issue:

    https://github.com/apache/spark/pull/8785
  
    So 2.0.0-preview is already out and we are in RC2 so I wouldn't expect any big changes happening right now if you want to take the time to update the PR :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48789707
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -37,6 +37,9 @@ private[sql] case class JDBCPartitioningInfo(
         numPartitions: Int)
     
     private[sql] object JDBCRelation {
    +
    +  import scala.collection.JavaConverters._
    --- End diff --
    
    OK, I'll merge after #10519 is closed, right now I see some changes that pretty much did what I did


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-162604858
  
    @srowen yeah, I'll reply shortly


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-168491449
  
    **[Test build #2301 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2301/consoleFull)** for PR 8785 at commit [`6a240b1`](https://github.com/apache/spark/commit/6a240b12ec994ddb989a22b46b5b90a3e9fdc014).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48714897
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala ---
    @@ -469,6 +469,13 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter with SharedSQLContext
         assert(derbyDialect.getJDBCType(BooleanType).map(_.databaseTypeDefinition).get == "BOOLEAN")
       }
     
    +  test("Spark-10625: JDBC read should allow driver to insert unserializable into properties") {
    --- End diff --
    
    Spark should be `SPARK` here, for consistency with other JIRA references.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Spark 10625

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r39701968
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -75,6 +76,19 @@ private[sql] object JDBCRelation {
         }
         ans.toArray
       }
    +
    +  def getEffectiveProperties(
    +                                        connectionProperties: Properties,
    --- End diff --
    
    The indentation here seems off (you may wish to run ./dev/lint-scala)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-167539588
  
    **[Test build #2258 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2258/consoleFull)** for PR 8785 at commit [`6af8fd8`](https://github.com/apache/spark/commit/6af8fd8b824f5f343a01868560b74a1f55acd02f).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-174732645
  
    Rebased with minimal changes to code style (and reverted whitespace changes as well as correcting import order). All tests passed. Let's finish this


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #8785: [Spark-10625] [SQL] Spark SQL JDBC read/write is unable t...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the issue:

    https://github.com/apache/spark/pull/8785
  
    Hi Reynold,
    
    Yes, by 1.6.2 it made some of the JDBC drivers (notably the one for SAP 
    HANA) to malfunction.
    
    The fix is easy, though I haven't test if its already fixed in 2.0+.
    
    Yours Peng
    
    
    On 2016-12-07 12:04 AM, Reynold Xin wrote:
    >
    > @tribbloid <https://github.com/tribbloid> is this a problem that needs 
    > to be fixed?
    >
    > \u2014
    > You are receiving this because you were mentioned.
    > Reply to this email directly, view it on GitHub 
    > <https://github.com/apache/spark/pull/8785#issuecomment-265356776>, or 
    > mute the thread 
    > <https://github.com/notifications/unsubscribe-auth/ADSDBd9p7cLc8MmfVv9-6WBhcwHpsz-2ks5rFj5cgaJpZM4F-xmq>.
    >
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r51036047
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala ---
    @@ -108,6 +122,6 @@ private[sql] case class JDBCRelation(
       override def insert(data: DataFrame, overwrite: Boolean): Unit = {
         data.write
           .mode(if (overwrite) SaveMode.Overwrite else SaveMode.Append)
    -      .jdbc(url, table, properties)
    +      .jdbc(url, table, JDBCRelation.getEffectiveProperties(properties))
    --- End diff --
    
    yes, reverted without breaking any test.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Spark-10625: Spark SQL JDBC read/write is unab...

Posted by holdenk <gi...@git.apache.org>.
Github user holdenk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r39886861
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala ---
    @@ -151,4 +150,13 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter {
         assert(2 === sqlContext.read.jdbc(url1, "TEST.PEOPLE1", properties).count)
         assert(2 === sqlContext.read.jdbc(url1, "TEST.PEOPLE1", properties).collect()(0).length)
       }
    +
    +  test("INSERT to JDBC Datasource with Unserializable Driver Properties") {
    +
    --- End diff --
    
    extra blank line


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: Spark-10625: Spark SQL JDBC read/write is unab...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-141598082
  
    all fixed (except the PR, will fix soon), thanks a lot for pointing them out!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-168313849
  
    Weird, OK. This LGTM.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-162703442
  
    @srowen for you second question: the connection property is deep copied twice to ensure that the original object is immutable. Reverting it breaks the scenario where a property is used in 2 JDBC write in short sequence. I haven't include this scenario into unit test yet, do you prefer me doing it? or this is not expected?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48489063
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.scala ---
    @@ -0,0 +1,53 @@
    +/*
    + * 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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  import scala.collection.JavaConverters._
    --- End diff --
    
    Patches should target specific problem and minimize collateral impact. No? Then I won't import utility package in upcoming pull requests.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by tribbloid <gi...@git.apache.org>.
Github user tribbloid commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-167461046
  
    All issues should be fixed by now, submitting final pull request


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8785#discussion_r48472434
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/jdbc/UnserializableDriverHelper.scala ---
    @@ -0,0 +1,53 @@
    +/*
    + * 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.jdbc
    +
    +import java.sql.{DriverManager, Connection}
    +import java.util.Properties
    +import java.util.logging.Logger
    +
    +object UnserializableDriverHelper {
    +
    +  import scala.collection.JavaConverters._
    --- End diff --
    
    This is imported locally in  a few places, why? Below you don't import org.h2.Driver though. I'm not worried about changing it though.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [Spark-10625] [SQL] Spark SQL JDBC read/write ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8785#issuecomment-168494148
  
    **[Test build #2304 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2304/consoleFull)** for PR 8785 at commit [`6a240b1`](https://github.com/apache/spark/commit/6a240b12ec994ddb989a22b46b5b90a3e9fdc014).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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