spark-instrumented-optimizer/python
hyukjinkwon aa4cf2b19e [SPARK-22651][PYTHON][ML] Prevent initiating multiple Hive clients for ImageSchema.readImages
## What changes were proposed in this pull request?

Calling `ImageSchema.readImages` multiple times as below in PySpark shell:

```python
from pyspark.ml.image import ImageSchema
data_path = 'data/mllib/images/kittens'
_ = ImageSchema.readImages(data_path, recursive=True, dropImageFailures=True).collect()
_ = ImageSchema.readImages(data_path, recursive=True, dropImageFailures=True).collect()
```

throws an error as below:

```
...
org.datanucleus.exceptions.NucleusDataStoreException: Unable to open a test connection to the given database. JDBC url = jdbc:derby:;databaseName=metastore_db;create=true, username = APP. Terminating connection pool (set lazyInit to true if you expect to start your database after your app). Original Exception: ------
java.sql.SQLException: Failed to start database 'metastore_db' with class loader org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1742f639f, see the next exception for details.
...
	at org.apache.derby.jdbc.AutoloadedDriver.connect(Unknown Source)
...
	at org.apache.hadoop.hive.metastore.HiveMetaStore.newRetryingHMSHandler(HiveMetaStore.java:5762)
...
	at org.apache.spark.sql.hive.client.HiveClientImpl.newState(HiveClientImpl.scala:180)
...
	at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$databaseExists$1.apply$mcZ$sp(HiveExternalCatalog.scala:195)
	at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$databaseExists$1.apply(HiveExternalCatalog.scala:195)
	at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$databaseExists$1.apply(HiveExternalCatalog.scala:195)
	at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:97)
	at org.apache.spark.sql.hive.HiveExternalCatalog.databaseExists(HiveExternalCatalog.scala:194)
	at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:100)
	at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:88)
	at org.apache.spark.sql.hive.HiveSessionStateBuilder.externalCatalog(HiveSessionStateBuilder.scala:39)
	at org.apache.spark.sql.hive.HiveSessionStateBuilder.catalog$lzycompute(HiveSessionStateBuilder.scala:54)
	at org.apache.spark.sql.hive.HiveSessionStateBuilder.catalog(HiveSessionStateBuilder.scala:52)
	at org.apache.spark.sql.hive.HiveSessionStateBuilder$$anon$1.<init>(HiveSessionStateBuilder.scala:69)
	at org.apache.spark.sql.hive.HiveSessionStateBuilder.analyzer(HiveSessionStateBuilder.scala:69)
	at org.apache.spark.sql.internal.BaseSessionStateBuilder$$anonfun$build$2.apply(BaseSessionStateBuilder.scala:293)
	at org.apache.spark.sql.internal.BaseSessionStateBuilder$$anonfun$build$2.apply(BaseSessionStateBuilder.scala:293)
	at org.apache.spark.sql.internal.SessionState.analyzer$lzycompute(SessionState.scala:79)
	at org.apache.spark.sql.internal.SessionState.analyzer(SessionState.scala:79)
	at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:70)
	at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:68)
	at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:51)
	at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:70)
	at org.apache.spark.sql.SparkSession.internalCreateDataFrame(SparkSession.scala:574)
	at org.apache.spark.sql.SparkSession.createDataFrame(SparkSession.scala:593)
	at org.apache.spark.sql.SparkSession.createDataFrame(SparkSession.scala:348)
	at org.apache.spark.sql.SparkSession.createDataFrame(SparkSession.scala:348)
	at org.apache.spark.ml.image.ImageSchema$$anonfun$readImages$2$$anonfun$apply$1.apply(ImageSchema.scala:253)
...
Caused by: ERROR XJ040: Failed to start database 'metastore_db' with class loader org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1742f639f, see the next exception for details.
	at org.apache.derby.iapi.error.StandardException.newException(Unknown Source)
	at org.apache.derby.impl.jdbc.SQLExceptionFactory.wrapArgsForTransportAcrossDRDA(Unknown Source)
	... 121 more
Caused by: ERROR XSDB6: Another instance of Derby may have already booted the database /.../spark/metastore_db.
...
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/ml/image.py", line 190, in readImages
    dropImageFailures, float(sampleRatio), seed)
  File "/.../spark/python/lib/py4j-0.10.6-src.zip/py4j/java_gateway.py", line 1160, in __call__
  File "/.../spark/python/pyspark/sql/utils.py", line 69, in deco
    raise AnalysisException(s.split(': ', 1)[1], stackTrace)
pyspark.sql.utils.AnalysisException: u'java.lang.RuntimeException: java.lang.RuntimeException: Unable to instantiate org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient;'
```

Seems we better stick to `SparkSession.builder.getOrCreate()` like:

51620e288b/python/pyspark/sql/streaming.py (L329)

dc5d34d8dc/python/pyspark/sql/column.py (L541)

33d43bf1b6/python/pyspark/sql/readwriter.py (L105)

## How was this patch tested?

This was tested as below in PySpark shell:

```python
from pyspark.ml.image import ImageSchema
data_path = 'data/mllib/images/kittens'
_ = ImageSchema.readImages(data_path, recursive=True, dropImageFailures=True).collect()
_ = ImageSchema.readImages(data_path, recursive=True, dropImageFailures=True).collect()
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19845 from HyukjinKwon/SPARK-22651.
2017-12-02 11:55:43 +09:00
..
docs [SPARK-21866][ML][PYSPARK] Adding spark image reader 2017-11-22 15:45:45 -08:00
lib [SPARK-21278][PYSPARK] Upgrade to Py4J 0.10.6 2017-07-05 16:33:23 -07:00
pyspark [SPARK-22651][PYTHON][ML] Prevent initiating multiple Hive clients for ImageSchema.readImages 2017-12-02 11:55:43 +09:00
test_support [SPARK-19610][SQL] Support parsing multiline CSV files 2017-02-28 13:34:33 -08:00
.gitignore [SPARK-3946] gitignore in /python includes wrong directory 2014-10-14 14:09:39 -07:00
MANIFEST.in [SPARK-18652][PYTHON] Include the example data and third-party licenses in pyspark package. 2016-12-07 06:09:27 +08:00
pylintrc [SPARK-13596][BUILD] Move misc top-level build files into appropriate subdirs 2016-03-07 14:48:02 -08:00
README.md [SPARK-21278][PYSPARK] Upgrade to Py4J 0.10.6 2017-07-05 16:33:23 -07:00
run-tests [SPARK-8583] [SPARK-5482] [BUILD] Refactor python/run-tests to integrate with dev/run-tests module system 2015-06-27 20:24:34 -07:00
run-tests.py [SPARK-14280][BUILD][WIP] Update change-version.sh and pom.xml to add Scala 2.12 profiles and enable 2.12 compilation 2017-09-01 19:21:21 +01:00
setup.cfg [SPARK-1267][SPARK-18129] Allow PySpark to be pip installed 2016-11-16 14:22:15 -08:00
setup.py [SPARK-22395][SQL][PYTHON] Fix the behavior of timestamp values for Pandas to respect session timezone 2017-11-28 16:45:22 +08:00

Apache Spark

Spark is a fast and general cluster computing system for Big Data. It provides high-level APIs in Scala, Java, Python, and R, and an optimized engine that supports general computation graphs for data analysis. It also supports a rich set of higher-level tools including Spark SQL for SQL and DataFrames, MLlib for machine learning, GraphX for graph processing, and Spark Streaming for stream processing.

http://spark.apache.org/

Online Documentation

You can find the latest Spark documentation, including a programming guide, on the project web page

Python Packaging

This README file only contains basic information related to pip installed PySpark. This packaging is currently experimental and may change in future versions (although we will do our best to keep compatibility). Using PySpark requires the Spark JARs, and if you are building this from source please see the builder instructions at "Building Spark".

The Python packaging for Spark is not intended to replace all of the other use cases. This Python packaged version of Spark is suitable for interacting with an existing cluster (be it Spark standalone, YARN, or Mesos) - but does not contain the tools required to setup your own standalone Spark cluster. You can download the full version of Spark from the Apache Spark downloads page.

NOTE: If you are using this with a Spark standalone cluster you must ensure that the version (including minor version) matches or you may experience odd errors.

Python Requirements

At its core PySpark depends on Py4J (currently version 0.10.6), but additional sub-packages have their own requirements (including numpy and pandas).