Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-22651][PYTHON][ML] Prevent initiating multiple Hive clients for ImageSchema.readImages #19845

Closed
wants to merge 3 commits into from

Conversation

HyukjinKwon
Copy link
Member

@HyukjinKwon HyukjinKwon commented Nov 29, 2017

What changes were proposed in this pull request?

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

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$1@742f639f, 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$1@742f639f, 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:

spark = SparkSession.builder.getOrCreate()

spark = SparkSession.builder.getOrCreate()

spark = SparkSession.builder.getOrCreate()

How was this patch tested?

This was tested as below in PySpark shell:

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()

@HyukjinKwon
Copy link
Member Author

I could fold this change into https://github.com/apache/spark/pull/19835. I only opened a separate PR here as it was easy to describe this issue separately and seems another issue although the change is small.

@HyukjinKwon
Copy link
Member Author

cc @jiangxb1987, @viirya who I am seeing touched and reviewed similar codes, and @imatiach-msft who's the primary author of this codes.

@SparkQA
Copy link

SparkQA commented Nov 29, 2017

Test build #84297 has finished for PR 19845 at commit c0c3c48.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@viirya
Copy link
Member

viirya commented Nov 29, 2017

Let me look into this tomorrow. :)

Copy link
Contributor

@jiangxb1987 jiangxb1987 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should be a valid fix, LGTM. Also cc @ueshin

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, LGTM.

Copy link
Member

@viirya viirya left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM

ctx = SparkContext._active_spark_context
spark = SparkSession(ctx)
image_schema = ctx._jvm.org.apache.spark.ml.image.ImageSchema
spark = SparkSession.builder.getOrCreate()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, I think this should be best practice to initialize SparkSession.

@HyukjinKwon
Copy link
Member Author

Thanks for reviewing this @jiangxb1987, @dongjoon-hyun and @viirya.

@viirya
Copy link
Member

viirya commented Nov 30, 2017

Shall we also add a simple test to ImageReaderTest?

@imatiach-msft
Copy link
Contributor

LGTM as long as all tests pass - but what about the other methods that use "ctx = SparkContext._active_spark_context" -- should those be modified as well? Can you run those one after the other in sequence too, or do they fail with the same error? I recall I tried to get the current session just like this in the first iteration but the python tests started to fail when run in parallel, that was the original reason to move to use _active_spark_context.

@HyukjinKwon
Copy link
Member Author

@imatiach-msft, ah, I think it's not about SparkContext but SparkSession, (SparkSession(...) directly) to be more clear, which seems causing multiple Hive clients when Hive support is enabled.

I think the most of PySpark's unit tests do not enable Hive's support by default. Also, I double checked and I ran this multiple times and it seems fine after this fix as well.

Let me keep my eyes on Jenkins tests about this. Basically the tests are ran in parallel on Jenkins.

@HyukjinKwon
Copy link
Member Author

HyukjinKwon commented Nov 30, 2017

@viirya, actually, I think it's not that simple to just add multiple ImageSchema.readImages (if you thought this way). I think I need another class with a Hive support enabled session if we should add the test. Let me try to add it now that you pointed it out anyway.

@SparkQA
Copy link

SparkQA commented Nov 30, 2017

Test build #84342 has finished for PR 19845 at commit 2e4c402.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class ImageReaderTest2(PySparkTestCase):

cls.tearDownClass()
raise unittest.SkipTest("Hive is not available")
cls.spark = HiveContext._createForTesting(cls.sc)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add classmethod tearDownClass to stop the cls.spark? I didn't see HiveContextSQLTests closes it anyway, maybe we can fix it too.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, that should be safer but let me fix it this one here only for now. Let's fix that up too when we touch some codes around there.

@viirya
Copy link
Member

viirya commented Dec 1, 2017

@HyukjinKwon Thanks. I forgot the Hive support is needed to test it. The added test looks good.

@SparkQA
Copy link

SparkQA commented Dec 1, 2017

Test build #84369 has finished for PR 19845 at commit 0c7e537.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member Author

Merged to master.

Thanks for reviewing this @viirya, @jiangxb1987, @dongjoon-hyun and @imatiach-msft.

@asfgit asfgit closed this in aa4cf2b Dec 2, 2017
@HyukjinKwon HyukjinKwon deleted the SPARK-22651 branch January 2, 2018 03:37
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
6 participants