From issues-return-197226-archive-asf-public=cust-asf.ponee.io@spark.apache.org Thu Jul 26 19:33:07 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 49774180621 for ; Thu, 26 Jul 2018 19:33:06 +0200 (CEST) Received: (qmail 58920 invoked by uid 500); 26 Jul 2018 17:33:05 -0000 Mailing-List: contact issues-help@spark.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@spark.apache.org Received: (qmail 58911 invoked by uid 99); 26 Jul 2018 17:33:05 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 26 Jul 2018 17:33:05 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id ED0F91808B1 for ; Thu, 26 Jul 2018 17:33:04 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -109.501 X-Spam-Level: X-Spam-Status: No, score=-109.501 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, KAM_ASCII_DIVIDERS=0.8, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, USER_IN_DEF_SPF_WL=-7.5, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id lBCLoyaWY1Ms for ; Thu, 26 Jul 2018 17:33:01 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id 708BA5F42E for ; Thu, 26 Jul 2018 17:33:01 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id D7258E25D4 for ; Thu, 26 Jul 2018 17:33:00 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 5809227765 for ; Thu, 26 Jul 2018 17:33:00 +0000 (UTC) Date: Thu, 26 Jul 2018 17:33:00 +0000 (UTC) From: "Cheng Lian (JIRA)" To: issues@spark.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (SPARK-24927) The hadoop-provided profile doesn't play well with Snappy-compressed Parquet files MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/SPARK-24927?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Cheng Lian updated SPARK-24927: ------------------------------- Description: Reproduction: {noformat} wget https://archive.apache.org/dist/spark/spark-2.3.1/spark-2.3.1-bin-without-hadoop.tgz wget https://archive.apache.org/dist/hadoop/core/hadoop-2.7.3/hadoop-2.7.3.tar.gz tar xzf spark-2.3.1-bin-without-hadoop.tgz tar xzf hadoop-2.7.3.tar.gz export SPARK_DIST_CLASSPATH=$(hadoop-2.7.3/bin/hadoop classpath) ./spark-2.3.1-bin-without-hadoop/bin/spark-shell --master local ... scala> spark.range(1).repartition(1).write.mode("overwrite").parquet("file:///tmp/test.parquet") {noformat} Exception: {noformat} Driver stacktrace: at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1602) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1590) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1589) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1589) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:831) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:831) at scala.Option.foreach(Option.scala:257) at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:831) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1823) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1772) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1761) at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:642) at org.apache.spark.SparkContext.runJob(SparkContext.scala:2034) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:194) ... 69 more Caused by: org.apache.spark.SparkException: Task failed while writing rows. at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:285) 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) Caused by: 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:1414) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:272) ... 8 more {noformat} Root cause: # Spark 2.3 [requires snappy-java 1.1.2.6 explicitly|https://github.com/apache/spark/blob/v2.3.1/pom.xml#L163] in the root POM. # However, the scope of snappy-java is set to {{$\{hadoop.deps.scope}}}, which is [set to {{provided}}|https://github.com/apache/spark/blob/v2.3.1/assembly/pom.xml#L224] when the {{hadoop-provided}} profile is enabled. Therefore, snappy-java 1.1.2.6 is not included in the pre-built {{spark-2.3.1-bin-without-hadoop.tgz}} package. Instead, snappy-java 1.0.4.1, which is bundled with Hadoop 2.3.1, is used in the reproduction above and caused the link error. I think we can simply remove [this line|https://github.com/apache/spark/blob/v2.3.1/pom.xml#L163] to get this issue fixed. was: Reproduction: {noformat} wget https://archive.apache.org/dist/spark/spark-2.3.1/spark-2.3.1-bin-without-hadoop.tgz wget https://archive.apache.org/dist/hadoop/core/hadoop-2.7.3/hadoop-2.7.3.tar.gz tar xzf spark-2.3.1-bin-without-hadoop.tgz tar xzf hadoop-2.7.3.tar.gz export SPARK_DIST_CLASSPATH=$(hadoop-2.7.3/bin/hadoop classpath) ./spark-2.3.1-bin-without-hadoop/bin/spark-shell --master local ... scala> spark.range(1).repartition(1).write.mode("overwrite").parquet("file:///tmp/test.parquet") {noformat} Exception: {noformat} Driver stacktrace: at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1602) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1590) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1589) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1589) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:831) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:831) at scala.Option.foreach(Option.scala:257) at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:831) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1823) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1772) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1761) at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:642) at org.apache.spark.SparkContext.runJob(SparkContext.scala:2034) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:194) ... 69 more Caused by: org.apache.spark.SparkException: Task failed while writing rows. at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:285) 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) Caused by: 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:1414) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:272) ... 8 more {noformat} Root cause: # Spark 2.3 [requires snappy-java 1.1.2.6 explicitly|https://github.com/apache/spark/blob/v2.3.1/pom.xml#L163] in the root POM. # However, the scope of snappy-java is set to {{$\{hadoop.deps.scope}}}, which is [set to {{provided}}|https://github.com/apache/spark/blob/v2.3.1/assembly/pom.xml#L224] when the {{hadoop-provided}} profile is enabled. Therefore, snappy-java 1.1.2.6 is not included in the pre-built {{spark-2.3.1-bin-without-hadoop.tgz}} package. Instead, snappy-java 1.0.4.1, which is bundled with Hadoop 2.3.1, is used in the reproduction above and caused the link error. I think we can simply remove [this line|https://github.com/apache/spark/blob/v2.3.1/pom.xml#L163] to get this issue fixed. > The hadoop-provided profile doesn't play well with Snappy-compressed Parquet files > ---------------------------------------------------------------------------------- > > Key: SPARK-24927 > URL: https://issues.apache.org/jira/browse/SPARK-24927 > Project: Spark > Issue Type: Bug > Components: Build > Affects Versions: 2.3.1, 2.3.2 > Reporter: Cheng Lian > Priority: Major > > Reproduction: > {noformat} > wget https://archive.apache.org/dist/spark/spark-2.3.1/spark-2.3.1-bin-without-hadoop.tgz > wget https://archive.apache.org/dist/hadoop/core/hadoop-2.7.3/hadoop-2.7.3.tar.gz > tar xzf spark-2.3.1-bin-without-hadoop.tgz > tar xzf hadoop-2.7.3.tar.gz > export SPARK_DIST_CLASSPATH=$(hadoop-2.7.3/bin/hadoop classpath) > ./spark-2.3.1-bin-without-hadoop/bin/spark-shell --master local > ... > scala> spark.range(1).repartition(1).write.mode("overwrite").parquet("file:///tmp/test.parquet") > {noformat} > Exception: > {noformat} > Driver stacktrace: > at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1602) > at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1590) > at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1589) > at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) > at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1589) > at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:831) > at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:831) > at scala.Option.foreach(Option.scala:257) > at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:831) > at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1823) > at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1772) > at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1761) > at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) > at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:642) > at org.apache.spark.SparkContext.runJob(SparkContext.scala:2034) > at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:194) > ... 69 more > Caused by: org.apache.spark.SparkException: Task failed while writing rows. > at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:285) > 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) > Caused by: 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:1414) > at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:272) > ... 8 more > {noformat} > Root cause: > # Spark 2.3 [requires snappy-java 1.1.2.6 explicitly|https://github.com/apache/spark/blob/v2.3.1/pom.xml#L163] in the root POM. > # However, the scope of snappy-java is set to {{$\{hadoop.deps.scope}}}, which is [set to {{provided}}|https://github.com/apache/spark/blob/v2.3.1/assembly/pom.xml#L224] when the {{hadoop-provided}} profile is enabled. > Therefore, snappy-java 1.1.2.6 is not included in the pre-built {{spark-2.3.1-bin-without-hadoop.tgz}} package. Instead, snappy-java 1.0.4.1, which is bundled with Hadoop 2.3.1, is used in the reproduction above and caused the link error. > I think we can simply remove [this line|https://github.com/apache/spark/blob/v2.3.1/pom.xml#L163] to get this issue fixed. -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org For additional commands, e-mail: issues-help@spark.apache.org