+dev@iceberg.apache.org. That's the correct list now.

Akshita,

The latest master doesn't yet have the fix that removes the conflict with
Spark's Parquet version. That fix is in PR #63
<https://github.com/apache/incubator-iceberg/pull/63>. If you build that
PR, then you should be able to shade everything in org.apache.parquet to
avoid the conflict with Spark.

The error you're getting does look like the one Manish hit. I think that is
because Spark can't match the column that you're trying to project by name.
Make sure the case you're using matches the case used in the table
definition because Iceberg is currently case sensitive. Xabriel is working
on fixing case sensitivity or at least making it optional for expressions
in PR #89 <https://github.com/apache/incubator-iceberg/pull/89>. If it
turns out to be case sensitivity, could you please open an issue on the
project and we'll fix it before the release?

rb

On Tue, Feb 5, 2019 at 3:02 PM Akshita Gupta <
akshita.gu...@orbitalinsight.com> wrote:

> Forwarding to iceberg developers. Not sure if dev@iceberg is valid.
>
> Begin forwarded message:
>
> *From: *Akshita Gupta <akshita.gu...@orbitalinsight.com>
> *Subject: **Quick question on writing parquet files to iceberg*
> *Date: *February 5, 2019 at 12:09:14 PM PST
> *To: *dev@iceberg.apache.org
>
> Hello
>
> Hello, I am having issue reading from partitioned parquet file in iceberg.
> I am able to do csv easily. I get the following error at .save(location):
>
> Exception in thread "main" java.lang.NullPointerException at
> com.netflix.iceberg.types.ReassignIds.field(ReassignIds.java:74) at
> com.netflix.iceberg.types.ReassignIds.field(ReassignIds.java:25) at
> com.netflix.iceberg.types.TypeUtil$VisitFieldFuture.get(TypeUtil.java:308)
> at com.google.common.collect.Iterators$8.next(Iterators.java:812) at
> com.google.common.collect.Lists.newArrayList(Lists.java:139) at
> com.google.common.collect.Lists.newArrayList(Lists.java:119) at
> com.netflix.iceberg.types.ReassignIds.struct(ReassignIds.java:52) at
> com.netflix.iceberg.types.ReassignIds.struct(ReassignIds.java:25) at
> com.netflix.iceberg.types.TypeUtil.visit(TypeUtil.java:341) at
> com.netflix.iceberg.types.TypeUtil$VisitFuture.get(TypeUtil.java:293) at
> com.netflix.iceberg.types.ReassignIds.schema(ReassignIds.java:37) at
> com.netflix.iceberg.types.ReassignIds.schema(ReassignIds.java:25) at
> com.netflix.iceberg.types.TypeUtil.visit(TypeUtil.java:313) at
> com.netflix.iceberg.types.TypeUtil.reassignIds(TypeUtil.java:122) at
> com.netflix.iceberg.spark.SparkSchemaUtil.convert(SparkSchemaUtil.java:163)
> at
> com.netflix.iceberg.spark.source.IcebergSource.createWriter(IcebergSource.java:67)
> at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:254) at
> org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:225) at
> com.mkyong.hashing.App.main(App.java:115).
>
> I remember Ryan mentioning to use the bundled version of iceberg earlier
> but I am not sure how to use that? I am partitioning in Java and added
> dependency using pom.xml.
>
> Here is my dependency xml. What else needs to be done to be able to read
> parquet?
>
> <?xml version="1.0" encoding="UTF-8"?>
> <project xmlns="http://maven.apache.org/POM/4.0.0";
>          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance";
>          xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
> http://maven.apache.org/xsd/maven-4.0.0.xsd";>
>     <modelVersion>4.0.0</modelVersion>
>
>     <groupId>iceberg-sample</groupId>
>     <artifactId>java-iceberg</artifactId>
>     <version>1.0-SNAPSHOT</version>
>
>     <properties>
>         <!-- https://maven.apache.org/general.html#encoding-warning -->
>         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
>
>         <maven.compiler.source>1.8</maven.compiler.source>
>         <maven.compiler.target>1.8</maven.compiler.target>
>     </properties>
>
>     <repositories>
>         <repository>
>             <id>jitpack.io</id>
>             <url>https://jitpack.io</url>
>         </repository>
>     </repositories>
>     <build>
>         <plugins>
>             <plugin>
>                 <groupId>org.apache.maven.plugins</groupId>
>                 <artifactId>maven-jar-plugin</artifactId>
>                 <configuration>
>                     <archive>
>                         <manifest>
>                             <mainClass>com.mkyong.hashing.App</mainClass>
>                         </manifest>
>                     </archive>
>                 </configuration>
>             </plugin>
>             <plugin>
>                 <groupId>org.apache.maven.plugins</groupId>
>                 <artifactId>maven-shade-plugin</artifactId>
>                 <version>3.2.0</version>
>                 <executions>
>                     <!-- Attach the shade goal into the package phase -->
>                     <execution>
>                         <phase>package</phase>
>                         <goals>
>                             <goal>shade</goal>
>                         </goals>
>                         <configuration>
>                             <transformers>
>                                 <transformer 
> implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
>                             </transformers>
>                         </configuration>
>                     </execution>
>                 </executions>
>             </plugin>
>         </plugins>
>     </build>
>
>
>     <dependencies>
>         <dependency>
>             <groupId>org.apache.spark</groupId>
>             <artifactId>spark-core_2.11</artifactId>
>             <version>2.3.2</version>
>         </dependency>
>         <dependency>
>             <groupId>org.apache.spark</groupId>
>             <artifactId>spark-sql_2.11</artifactId>
>             <version>2.3.2</version>
>         </dependency>
>         <dependency>
>             <groupId>com.amazonaws</groupId>
>             <artifactId>aws-java-sdk</artifactId>
>             <version>1.11.417</version>
>         </dependency>
>         <dependency>
>             <groupId>com.github.Netflix.iceberg</groupId>
>             <artifactId>iceberg-spark</artifactId>
>             <version>0.6.2</version>
>         </dependency>
>
>         <dependency>
>             <groupId>com.github.Netflix.iceberg</groupId>
>             <artifactId>iceberg-common</artifactId>
>             <version>0.6.2</version>
>         </dependency>
>         <dependency>
>             <groupId>com.github.Netflix.iceberg</groupId>
>             <artifactId>iceberg-api</artifactId>
>             <version>0.6.2</version>
>         </dependency>
>         <dependency>
>             <groupId>com.github.Netflix.iceberg</groupId>
>             <artifactId>iceberg-core</artifactId>
>             <version>0.6.2</version>
>         </dependency>
>         <dependency>
>             <groupId>com.github.Netflix.iceberg</groupId>
>             <artifactId>iceberg-parquet</artifactId>
>             <version>0.6.2</version>
>         </dependency>
>
>         <dependency>
>             <groupId>org.apache.hadoop</groupId>
>             <artifactId>hadoop-aws</artifactId>
>             <version>3.1.1</version>
>         </dependency>
>         <!--FATAL: DO NOT ADD THIS-->
>         <!--<dependency>-->
>         <!--<groupId>org.apache.hadoop</groupId>-->
>         <!--<artifactId>hadoop-hdfs</artifactId>-->
>         <!--<version>3.1.1</version>-->
>         <!--</dependency>-->
>         <dependency>
>             <groupId>org.apache.hadoop</groupId>
>             <artifactId>hadoop-common</artifactId>
>             <version>3.1.1</version>
>         </dependency>
>         <dependency>
>             <groupId>org.apache.parquet</groupId>
>             <artifactId>parquet-common</artifactId>
>             <version>1.10.0</version>
>         </dependency>
>         <dependency>
>             <groupId>org.apache.parquet</groupId>
>             <artifactId>parquet-column</artifactId>
>             <version>1.10.0</version>
>         </dependency>
>         <dependency>
>             <groupId>org.apache.parquet</groupId>
>             <artifactId>parquet-hadoop</artifactId>
>             <version>1.10.0</version>
>         </dependency>
>         <dependency>
>             <groupId>org.apache.avro</groupId>
>             <artifactId>avro</artifactId>
>             <version>1.8.2</version>
>         </dependency>
>         <dependency>
>             <groupId>com.googlecode.json-simple</groupId>
>             <artifactId> json-simple</artifactId>
>             <version>1.1</version>
>         </dependency>
>     </dependencies>
> </project>
>
>
>
>
> --
> 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 iceberg-devel+unsubscr...@googlegroups.com.
> To post to this group, send email to iceberg-de...@googlegroups.com.
> To view this discussion on the web visit
> https://groups.google.com/d/msgid/iceberg-devel/31CCFFC7-913C-4CA6-882C-155C47B22DF6%40orbitalinsight.com
> <https://groups.google.com/d/msgid/iceberg-devel/31CCFFC7-913C-4CA6-882C-155C47B22DF6%40orbitalinsight.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