[ https://issues.apache.org/jira/browse/SPARK-24018?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16537551#comment-16537551 ]
Patrick Clay commented on SPARK-24018: -------------------------------------- I believe we are both partially correct in that a fix (with Spark 2.3.0) does require snappy-java-1.1.2, and it was caused by SPARK-18646. The native library loader of Snappy 1.0.4 [uses a self-described hack|https://github.com/xerial/snappy-java/blob/snappy-java-1.0.4/src/main/java/org/xerial/snappy/SnappyLoader.java#L175] to inject the loader onto the root class loader. The hack was [later removed|https://github.com/xerial/snappy-java/commit/06f007a08#diff-a1c8fc77f8] in 1.1.2, which allows the non-inheriting class loader to pick it up. I believed this only affects spark-shell, because neither pyspark (the REPL and spark-submit) nor {code:java} ./bin/spark-submit --class org.apache.spark.examples.sql.SQLDataSourceExample examples/jars/spark-examples_2.11-2.3.0.jar{code} have this issue. What repro did you have without spark-shell? I don't believe this is related to Parquet versioning because this also throws: {code:java} scala> import org.xerial.snappy.Snappy import org.xerial.snappy.Snappy scala> sc.parallelize(Seq("foo")).map(Snappy.compress).collect 2018-07-09 13:44:14 ERROR Executor:91 - Exception in task 11.0 in stage 0.0 (TID 11) java.lang.UnsatisfiedLinkError: org.xerial.snappy.SnappyNative.maxCompressedLength(I)I ...{code} In answer to your last question I did not pass any arguments to spark-shell. All I did to repro was {code:java} export SPARK_DIST_CLASSPATH=$(~/Downloads/hadoop-2.8.3/bin/hadoop classpath) ~/Downloads/spark-2.3.0-bin-without-hadoop/bin/spark-shell{code} > Spark-without-hadoop package fails to create or read parquet files with > snappy compression > ------------------------------------------------------------------------------------------ > > Key: SPARK-24018 > URL: https://issues.apache.org/jira/browse/SPARK-24018 > Project: Spark > Issue Type: Bug > Components: Deploy > Affects Versions: 2.3.0 > Reporter: Jean-Francis Roy > Priority: Minor > > On a brand-new installation of Spark 2.3.0 with a user-provided hadoop-2.8.3, > Spark fails to read or write dataframes in parquet format with snappy > compression. > This is due to an incompatibility between the snappy-java version that is > required by parquet (parquet is provided in Spark jars but snappy isn't) and > the version that is available from hadoop-2.8.3. > > Steps to reproduce: > * Download and extract hadoop-2.8.3 > * Download and extract spark-2.3.0-without-hadoop > * export JAVA_HOME, HADOOP_HOME, SPARK_HOME, PATH > * Following instructions from > [https://spark.apache.org/docs/latest/hadoop-provided.html], set > SPARK_DIST_CLASSPATH=$(hadoop classpath) in spark-env.sh > * Start a spark-shell, enter the following: > > {code:java} > import spark.implicits._ > val df = List(1, 2, 3, 4).toDF > df.write > .format("parquet") > .option("compression", "snappy") > .mode("overwrite") > .save("test.parquet") > {code} > > > This fails with the following: > {noformat} > java.lang.UnsatisfiedLinkError: > org.xerial.snappy.SnappyNative.maxCompressedLength(I)I > at org.xerial.snappy.SnappyNative.maxCompressedLength(Native Method) > at org.xerial.snappy.Snappy.maxCompressedLength(Snappy.java:316) > at > org.apache.parquet.hadoop.codec.SnappyCompressor.compress(SnappyCompressor.java:67) > at > org.apache.hadoop.io.compress.CompressorStream.compress(CompressorStream.java:81) > at > org.apache.hadoop.io.compress.CompressorStream.finish(CompressorStream.java:92) > at > org.apache.parquet.hadoop.CodecFactory$BytesCompressor.compress(CodecFactory.java:112) > at > org.apache.parquet.hadoop.ColumnChunkPageWriteStore$ColumnChunkPageWriter.writePage(ColumnChunkPageWriteStore.java:93) > at > org.apache.parquet.column.impl.ColumnWriterV1.writePage(ColumnWriterV1.java:150) > at > org.apache.parquet.column.impl.ColumnWriterV1.flush(ColumnWriterV1.java:238) > at > org.apache.parquet.column.impl.ColumnWriteStoreV1.flush(ColumnWriteStoreV1.java:121) > at > org.apache.parquet.hadoop.InternalParquetRecordWriter.flushRowGroupToStore(InternalParquetRecordWriter.java:167) > at > org.apache.parquet.hadoop.InternalParquetRecordWriter.close(InternalParquetRecordWriter.java:109) > at > org.apache.parquet.hadoop.ParquetRecordWriter.close(ParquetRecordWriter.java:163) > at > org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.close(ParquetOutputWriter.scala:42) > at > org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.releaseResources(FileFormatWriter.scala:405) > at > org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.execute(FileFormatWriter.scala:396) > at > org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:269) > at > org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:267) > at > org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1411) > at > org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:272) > at > org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:197) > at > org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:196) > at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87) at > org.apache.spark.scheduler.Task.run(Task.scala:109) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at java.lang.Thread.run(Thread.java:748){noformat} > > Downloading snappy-java-1.1.2.6.jar and placing it in Sparks's jar folder > solves the issue. -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org