+ Iceberg dev list that was somehow dropped

I just looked at Spark and it turns out that the update to Parquet 1.10.0
<https://github.com/apache/spark/commit/cac9b1dea1bb44fa42abf77829c05bf93f70cf20>
happened after 2.3.x was released. Sorry about that, I thought it made it
into 2.3.0.

I can't really recommend replacing the Spark Jars because the update to
1.10.0 required some change to Spark's use of internal Parquet APIs. If you
can use 2.4.0, we just committed 2.4 support
<https://github.com/apache/incubator-iceberg/commit/dd009546eb291e19320aa3fe7bf49e7813c6a72d>
to master. Otherwise, I think it still works to read from other formats and
write to Parquet.

I'll also work on replacing the Spark write path with an Iceberg-native
one. The reason why we don't shade all of Parquet is that we use Spark's
writer. But the native writer should be less fiddly and faster.

rb

On Mon, Dec 10, 2018 at 7:56 AM Yevgeniy Viller <[email protected]>
wrote:

> We used iceberg-runtime Jar.
>
> On Friday, December 7, 2018 at 6:07:12 PM UTC-5, Ryan Blue wrote:
>>
>> Which Jar did you use? You should be using the iceberg-runtime Jar.
>>
>> On Fri, Dec 7, 2018 at 12:22 PM Yevgeniy Viller <[email protected]>
>> wrote:
>>
>>> Hey Ryan,
>>>
>>> Thank you for your response.
>>> We built the jar locally and tried it on EMR with Haddop 2.8.5 and Spark
>>> 2.3.2.
>>> We run into issue when tried to write data in to the table
>>> val df = spark.read.parquet("s3a://path/to/parquet/files")
>>>
>>> df.write.mode(org.apache.spark.sql.SaveMode.Append).format("iceberg").save("hdfs:///tmp//iceberg_table1")
>>>
>>> and
>>>
>>> got this error:
>>> df: org.apache.spark.sql.DataFrame = [source: struct<component: string,
>>> caller: string>, request_ids:
>>> array<struct<namespace:int,id:string,authentication_state:string,type:string,timestamp:bigint>>
>>> ... 58 more fields] org.apache.spark.SparkException: Writing job aborted.
>>> at
>>> org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec.doExecute(WriteToDataSourceV2.scala:112)
>>> at
>>> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
>>> at
>>> org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
>>> at
>>> org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
>>> at
>>> org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
>>> at
>>> org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
>>> at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127) at
>>> org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:80)
>>> at
>>> org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:80)
>>> at
>>> org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:656)
>>> at
>>> org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:656)
>>> at
>>> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:77)
>>> at
>>> org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:656)
>>> at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:256) at
>>> org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:225) ... 52
>>> elided Caused by: org.apache.spark.SparkException: Job aborted due to stage
>>> failure: Task 0 in stage 2.0 failed 4 times, most recent failure: Lost task
>>> 0.3 in stage 2.0 (TID 14, ip-172-18-0-38.qe2.demdex.com, executor 1):
>>> java.lang.NoSuchMethodError:
>>> org.apache.parquet.hadoop.ParquetWriter$Builder.<init>(Lorg/apache/parquet/io/OutputFile;)V
>>> at
>>> com.netflix.iceberg.parquet.Parquet$ParquetWriteBuilder.<init>(Parquet.java:206)
>>> at
>>> com.netflix.iceberg.parquet.Parquet$ParquetWriteBuilder.<init>(Parquet.java:199)
>>> at com.netflix.iceberg.parquet.Parquet$WriteBuilder.build(Parquet.java:187)
>>> at
>>> com.netflix.iceberg.spark.source.Writer$WriterFactory$SparkAppenderFactory.newAppender(Writer.java:292)
>>> at
>>> com.netflix.iceberg.spark.source.Writer$UnpartitionedWriter.<init>(Writer.java:331)
>>> at
>>> com.netflix.iceberg.spark.source.Writer$WriterFactory.createDataWriter(Writer.java:223)
>>> at
>>> org.apache.spark.sql.execution.datasources.v2.DataWritingSparkTask$.run(WriteToDataSourceV2.scala:129)
>>> at
>>> org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec$$anonfun$2.apply(WriteToDataSourceV2.scala:79)
>>> at
>>> org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec$$anonfun$2.apply(WriteToDataSourceV2.scala:78)
>>> 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) Driver stacktrace: at
>>> org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1803)
>>> at
>>> org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1791)
>>> at
>>> org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1790)
>>> 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:1790)
>>> at
>>> org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:871)
>>> at
>>> org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:871)
>>> at scala.Option.foreach(Option.scala:257) at
>>> org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:871)
>>> at
>>> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2024)
>>> at
>>> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1973)
>>> at
>>> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1962)
>>> at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) at
>>> org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:682) at
>>> org.apache.spark.SparkContext.runJob(SparkContext.scala:2034) at
>>> org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec.doExecute(WriteToDataSourceV2.scala:82)
>>> ... 66 more Caused by: java.lang.NoSuchMethodError:
>>> org.apache.parquet.hadoop.ParquetWriter$Builder.<init>(Lorg/apache/parquet/io/OutputFile;)V
>>> at
>>> com.netflix.iceberg.parquet.Parquet$ParquetWriteBuilder.<init>(Parquet.java:206)
>>> at
>>> com.netflix.iceberg.parquet.Parquet$ParquetWriteBuilder.<init>(Parquet.java:199)
>>> at com.netflix.iceberg.parquet.Parquet$WriteBuilder.build(Parquet.java:187)
>>> at
>>> com.netflix.iceberg.spark.source.Writer$WriterFactory$SparkAppenderFactory.newAppender(Writer.java:292)
>>> at
>>> com.netflix.iceberg.spark.source.Writer$UnpartitionedWriter.<init>(Writer.java:331)
>>> at
>>> com.netflix.iceberg.spark.source.Writer$WriterFactory.createDataWriter(Writer.java:223)
>>> at
>>> org.apache.spark.sql.execution.datasources.v2.DataWritingSparkTask$.run(WriteToDataSourceV2.scala:129)
>>> at
>>> org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec$$anonfun$2.apply(WriteToDataSourceV2.scala:79)
>>> at
>>> org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec$$anonfun$2.apply(WriteToDataSourceV2.scala:78)
>>> 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) ... 3
>>> more
>>>
>>> After we tried to run it on a local hadoop setup and run into the same
>>> issue.
>>> We tried different setups of Hadoop and Spark and still getting the same
>>> error.
>>>
>>> Can you please point us to the right direction here?
>>>
>>> Thanks,
>>> Yevgeniy
>>>
>>>
>>> On Tuesday, December 4, 2018 at 8:10:37 PM UTC-5
>>> <https://issues.apache.org/jira/browse/UTC-5>, Ryan Blue wrote:
>>>
>>>> Yevgeniy,
>>>> The project recently moved to Apache. I’m adding the new mailing list
>>>> and will update the old README with some pointers.
>>>>
>>>> The ASF site has newer Javadoc:
>>>> https://iceberg.apache.org/javadoc/0.6.0/index.html?com/netflix/iceberg/package-summary.html
>>>>
>>>> Right now, the easiest way to test is with a path-based table. That’s
>>>> what Spark supports, since we haven’t updated it to use the Hive metastore.
>>>> You can add Iceberg by downloading the iceberg-runtime Jar and dropping it
>>>> into your Spark classpath using --jars.
>>>>
>>>> Spark 2.3.x only supports interacting with Iceberg or other v2 sources
>>>> through the DataFrame API and doesn’t support creating tables with DDL.
>>>> We’re working on getting those features into Spark, but for now you have to
>>>> create a table and then write to it from Spark. Here’s an example:
>>>>
>>>> val schema = new Schema(...)
>>>> val spec = PartitionSpec.builderFor(schema).build() // add configuration 
>>>> for your partitioning
>>>> val tables = new HadoopTables(spark.sparkContext.hadoopConfiguration)
>>>> val table = tables.create(schema, spec, "hdfs://nn:8020/path/to/table")
>>>>
>>>> Once your table is created, you can write to it using the dataframe
>>>> API. Be sure you sort the data frame to group data in each partition.
>>>>
>>>> df.write.format("iceberg").save("hdfs://nn:8020/path/to/table)
>>>>
>>>> Once data is written, you can read from the table like this:
>>>>
>>>> val df = spark.read.format("iceberg").load("hdfs://nn:8020/path/to/table")
>>>>
>>>> We will be adding Hive support so you can refer to your table by name
>>>> and use the Hive metastore to track its metadata, but hadoop tables should
>>>> get you started with your evaluation. Thanks for reaching out!
>>>>
>>>> rb
>>>>
>>>> On Tue, Dec 4, 2018 at 7:51 AM Yevgeniy Viller <[email protected]>
>>>> wrote:
>>>>
>>>
>>>>> Hey Ryan,
>>>>>
>>>>> We are doing POC of Iceberg against our internal datawarehouse
>>>>> platform.
>>>>> I read Spec docs and example in Git. However, it is still little hard
>>>>> to get started without proper examples. Also current version in Git is
>>>>> 0.5.1, but API docs
>>>>> https://docs.google.com/document/d/1Q-zL5lSCle6NEEdyfiYsXYzX_Q8Qf0ctMyGBKslOswA/edit#heading=h.vga9bjlv1x2e
>>>>> is for 0.3.0. Do you have link current version of APIs specs?
>>>>>
>>>>> Thanks,
>>>>> Yevgeniy
>>>>>
>>>>> On Thursday, January 4, 2018 at 2:19:20 PM UTC-5
>>>>> <https://issues.apache.org/jira/browse/UTC-5>, Ryan Blue wrote:
>>>>>>
>>>>>> The Iceberg repository is now public on github, here:
>>>>>> https://github.com/Netflix/iceberg
>>>>>>
>>>>>> The project is built with gradle and requires a Spark 2.3.0-SNAPSHOT
>>>>>> (for Datasource V2) and Parquet 1.9.1-SNAPSHOT (for API additions and bug
>>>>>> fixes).
>>>>>>
>>>>>> An early version of the spec is available for comments here:
>>>>>> https://docs.google.com/document/d/1Q-zL5lSCle6NEEdyfiYsXYzX_Q8Qf0ctMyGBKslOswA/edit?usp=sharing
>>>>>>
>>>>>> Feedback is welcome!
>>>>>>
>>>>>> rb
>>>>>>
>>>>>> --
>>>>>> Ryan Blue
>>>>>> Software Engineer
>>>>>> Netflix
>>>>>>
>>>>> --
>>>>> You received this message because you are subscribed to the Google
>>>>> Groups "Iceberg Developers" group.
>>>>>
>>>> To unsubscribe from this group and stop receiving emails from it, send
>>>>> an email to [email protected].
>>>>> To post to this group, send email to [email protected].
>>>>
>>>>
>>>>> To view this discussion on the web visit
>>>>> https://groups.google.com/d/msgid/iceberg-devel/68ae23cd-709d-40ba-a4d2-0f0c4413f27e%40googlegroups.com
>>>>> <https://groups.google.com/d/msgid/iceberg-devel/68ae23cd-709d-40ba-a4d2-0f0c4413f27e%40googlegroups.com?utm_medium=email&utm_source=footer>
>>>>> .
>>>>> For more options, visit https://groups.google.com/d/optout.
>>>>>
>>>>
>>>>
>>>> --
>>>> Ryan Blue
>>>> Software Engineer
>>>> Netflix
>>>>
>>> --
>>> You received this message because you are subscribed to the Google
>>> Groups "Iceberg Developers" group.
>>> To unsubscribe from this group and stop receiving emails from it, send
>>> an email to [email protected].
>>> To post to this group, send email to [email protected].
>>> To view this discussion on the web visit
>>> https://groups.google.com/d/msgid/iceberg-devel/29647721-3894-4cee-a7de-cc072afb6090%40googlegroups.com
>>> <https://groups.google.com/d/msgid/iceberg-devel/29647721-3894-4cee-a7de-cc072afb6090%40googlegroups.com?utm_medium=email&utm_source=footer>
>>> .
>>> For more options, visit https://groups.google.com/d/optout.
>>>
>> --
> You received this message because you are subscribed to the Google Groups
> "Iceberg Developers" group.
> To unsubscribe from this group and stop receiving emails from it, send an
> email to [email protected].
> To post to this group, send email to [email protected].
> To view this discussion on the web visit
> https://groups.google.com/d/msgid/iceberg-devel/2c66cb07-a399-47a9-95a7-172acd5602d8%40googlegroups.com
> <https://groups.google.com/d/msgid/iceberg-devel/2c66cb07-a399-47a9-95a7-172acd5602d8%40googlegroups.com?utm_medium=email&utm_source=footer>
> .
> For more options, visit https://groups.google.com/d/optout.
>


-- 
Ryan Blue
Software Engineer
Netflix

Reply via email to