spark-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Cheng Lian <lian.cs....@gmail.com>
Subject Re: Parquet Array Support Broken?
Date Tue, 08 Sep 2015 10:54:59 GMT
Yeah, this is a typical Parquet interoperability issue due to 
unfortunate historical reasons. Hive (actually parquet-hive) gives the 
following schema for array<int>:

message m0 {
   optional group f (LIST) {
     repeated group bag {
       optional int32 array_element;
}
}
}

while Spark SQL gives

message m1 {
   optional group f (LIST) {
     repeated group bag {
       optional int32 array;
     }
   }
}

So Spark 1.4 couldn't find the expected field "array" in the target 
Parquet file.  As Ruslan suggested, Spark 1.5 addresses this issue 
properly and is able to read Parquet files generated by most, if not 
all, Parquet data models out there.

You may find more details about Parquet interoperability in this post if 
you are interested 
https://www.mail-archive.com/user@spark.apache.org/msg35663.html

Cheng

On 9/8/15 6:19 AM, Alex Kozlov wrote:
> Thank you - it works if the file is created in Spark
>
> On Mon, Sep 7, 2015 at 3:06 PM, Ruslan Dautkhanov 
> <dautkhanov@gmail.com <mailto:dautkhanov@gmail.com>> wrote:
>
>     Read response from Cheng Lian <lian.cs.zju@gmail.com
>     <mailto:lian.cs.zju@gmail.com>> on Aug/27th - it looks the same
>     problem.
>
>     Workarounds
>     1. write that parquet file in Spark;
>     2. upgrade to Spark 1.5.
>
>     --
>     Ruslan Dautkhanov
>
>     On Mon, Sep 7, 2015 at 3:52 PM, Alex Kozlov <alexvk@gmail.com
>     <mailto:alexvk@gmail.com>> wrote:
>
>         No, it was created in Hive by CTAS, but any help is
>         appreciated...
>
>         On Mon, Sep 7, 2015 at 2:51 PM, Ruslan Dautkhanov
>         <dautkhanov@gmail.com <mailto:dautkhanov@gmail.com>> wrote:
>
>             That parquet table wasn't created in Spark, is it?
>
>             There was a recent discussion on this list that complex
>             data types in Spark prior to 1.5 often incompatible with
>             Hive for example, if I remember correctly.
>
>             On Mon, Sep 7, 2015, 2:57 PM Alex Kozlov <alexvk@gmail.com
>             <mailto:alexvk@gmail.com>> wrote:
>
>                 I am trying to read an (array typed) parquet file in
>                 spark-shell (Spark 1.4.1 with Hadoop 2.6):
>
>                 {code}
>                 $ bin/spark-shell
>                 log4j:WARN No appenders could be found for logger
>                 (org.apache.hadoop.metrics2.lib.MutableMetricsFactory).
>                 log4j:WARN Please initialize the log4j system properly.
>                 log4j:WARN See
>                 http://logging.apache.org/log4j/1.2/faq.html#noconfig
>                 for more info.
>                 Using Spark's default log4j profile:
>                 org/apache/spark/log4j-defaults.properties
>                 15/09/07 13:45:22 INFO SecurityManager: Changing view
>                 acls to: hivedata
>                 15/09/07 13:45:22 INFO SecurityManager: Changing
>                 modify acls to: hivedata
>                 15/09/07 13:45:22 INFO SecurityManager:
>                 SecurityManager: authentication disabled; ui acls
>                 disabled; users with view permissions: Set(hivedata);
>                 users with modify permissions: Set(hivedata)
>                 15/09/07 13:45:23 INFO HttpServer: Starting HTTP Server
>                 15/09/07 13:45:23 INFO Utils: Successfully started
>                 service 'HTTP class server' on port 43731.
>                 Welcome to
>                       ____        __
>                      / __/__  ___ _____/ /__
>                     _\ \/ _ \/ _ `/ __/  '_/
>                    /___/ .__/\_,_/_/ /_/\_\ version 1.4.1
>                       /_/
>
>                 Using Scala version 2.10.4 (Java HotSpot(TM) 64-Bit
>                 Server VM, Java 1.8.0)
>                 Type in expressions to have them evaluated.
>                 Type :help for more information.
>                 15/09/07 13:45:26 INFO SparkContext: Running Spark
>                 version 1.4.1
>                 15/09/07 13:45:26 INFO SecurityManager: Changing view
>                 acls to: hivedata
>                 15/09/07 13:45:26 INFO SecurityManager: Changing
>                 modify acls to: hivedata
>                 15/09/07 13:45:26 INFO SecurityManager:
>                 SecurityManager: authentication disabled; ui acls
>                 disabled; users with view permissions: Set(hivedata);
>                 users with modify permissions: Set(hivedata)
>                 15/09/07 13:45:27 INFO Slf4jLogger: Slf4jLogger started
>                 15/09/07 13:45:27 INFO Remoting: Starting remoting
>                 15/09/07 13:45:27 INFO Remoting: Remoting started;
>                 listening on addresses
>                 :[akka.tcp://sparkDriver@10.10.30.52:46083
>                 <http://sparkDriver@10.10.30.52:46083>]
>                 15/09/07 13:45:27 INFO Utils: Successfully started
>                 service 'sparkDriver' on port 46083.
>                 15/09/07 13:45:27 INFO SparkEnv: Registering
>                 MapOutputTracker
>                 15/09/07 13:45:27 INFO SparkEnv: Registering
>                 BlockManagerMaster
>                 15/09/07 13:45:27 INFO DiskBlockManager: Created local
>                 directory at
>                 /tmp/spark-f313315a-0769-4057-835d-196cfe140a26/blockmgr-bd1b8498-9f6a-47c4-ae59-8800563f97d0
>                 15/09/07 13:45:27 INFO MemoryStore: MemoryStore
>                 started with capacity 265.1 MB
>                 15/09/07 13:45:27 INFO HttpFileServer: HTTP File
>                 server directory is
>                 /tmp/spark-f313315a-0769-4057-835d-196cfe140a26/httpd-3fbe0c9d-c0c5-41ef-bf72-4f0ef59bfa21
>                 15/09/07 13:45:27 INFO HttpServer: Starting HTTP Server
>                 15/09/07 13:45:27 INFO Utils: Successfully started
>                 service 'HTTP file server' on port 38717.
>                 15/09/07 13:45:27 INFO SparkEnv: Registering
>                 OutputCommitCoordinator
>                 15/09/07 13:45:27 WARN Utils: Service 'SparkUI' could
>                 not bind on port 4040. Attempting port 4041.
>                 15/09/07 13:45:27 INFO Utils: Successfully started
>                 service 'SparkUI' on port 4041.
>                 15/09/07 13:45:27 INFO SparkUI: Started SparkUI at
>                 http://10.10.30.52:4041
>                 15/09/07 13:45:27 INFO Executor: Starting executor ID
>                 driver on host localhost
>                 15/09/07 13:45:27 INFO Executor: Using REPL class URI:
>                 http://10.10.30.52:43731
>                 15/09/07 13:45:27 INFO Utils: Successfully started
>                 service
>                 'org.apache.spark.network.netty.NettyBlockTransferService'
>                 on port 60973.
>                 15/09/07 13:45:27 INFO NettyBlockTransferService:
>                 Server created on 60973
>                 15/09/07 13:45:27 INFO BlockManagerMaster: Trying to
>                 register BlockManager
>                 15/09/07 13:45:27 INFO BlockManagerMasterEndpoint:
>                 Registering block manager localhost:60973 with 265.1
>                 MB RAM, BlockManagerId(driver, localhost, 60973)
>                 15/09/07 13:45:27 INFO BlockManagerMaster: Registered
>                 BlockManager
>                 15/09/07 13:45:28 INFO SparkILoop: Created spark context..
>                 Spark context available as sc.
>                 15/09/07 13:45:28 INFO HiveContext: Initializing
>                 execution hive, version 0.13.1
>                 15/09/07 13:45:28 INFO HiveMetaStore: 0: Opening raw
>                 store with implemenation
>                 class:org.apache.hadoop.hive.metastore.ObjectStore
>                 15/09/07 13:45:29 INFO ObjectStore: ObjectStore,
>                 initialize called
>                 15/09/07 13:45:29 INFO Persistence: Property
>                 hive.metastore.integral.jdo.pushdown unknown - will be
>                 ignored
>                 15/09/07 13:45:29 INFO Persistence: Property
>                 datanucleus.cache.level2 unknown - will be ignored
>                 15/09/07 13:45:29 WARN Connection: BoneCP specified
>                 but not present in CLASSPATH (or one of dependencies)
>                 15/09/07 13:45:29 WARN Connection: BoneCP specified
>                 but not present in CLASSPATH (or one of dependencies)
>                 15/09/07 13:45:36 INFO ObjectStore: Setting MetaStore
>                 object pin classes with
>                 hive.metastore.cache.pinobjtypes="Table,StorageDescriptor,SerDeInfo,Partition,Database,Type,FieldSchema,Order"
>                 15/09/07 13:45:36 INFO MetaStoreDirectSql: MySQL check
>                 failed, assuming we are not on mysql: Lexical error at
>                 line 1, column 5. Encountered: "@" (64), after : "".
>                 15/09/07 13:45:37 INFO Datastore: The class
>                 "org.apache.hadoop.hive.metastore.model.MFieldSchema"
>                 is tagged as "embedded-only" so does not have its own
>                 datastore table.
>                 15/09/07 13:45:37 INFO Datastore: The class
>                 "org.apache.hadoop.hive.metastore.model.MOrder" is
>                 tagged as "embedded-only" so does not have its own
>                 datastore table.
>                 15/09/07 13:45:42 INFO Datastore: The class
>                 "org.apache.hadoop.hive.metastore.model.MFieldSchema"
>                 is tagged as "embedded-only" so does not have its own
>                 datastore table.
>                 15/09/07 13:45:42 INFO Datastore: The class
>                 "org.apache.hadoop.hive.metastore.model.MOrder" is
>                 tagged as "embedded-only" so does not have its own
>                 datastore table.
>                 15/09/07 13:45:43 INFO ObjectStore: Initialized
>                 ObjectStore
>                 15/09/07 13:45:43 WARN ObjectStore: Version
>                 information not found in metastore.
>                 hive.metastore.schema.verification is not enabled so
>                 recording the schema version 0.13.1aa
>                 15/09/07 13:45:44 INFO HiveMetaStore: Added admin role
>                 in metastore
>                 15/09/07 13:45:44 INFO HiveMetaStore: Added public
>                 role in metastore
>                 15/09/07 13:45:44 INFO HiveMetaStore: No user is added
>                 in admin role, since config is empty
>                 15/09/07 13:45:45 INFO SessionState: No Tez session
>                 required at this point. hive.execution.engine=mr.
>                 15/09/07 13:45:45 INFO SparkILoop: Created sql context
>                 (with Hive support)..
>                 SQL context available as sqlContext.
>
>                 scala> val df =
>                 sqlContext.parquetFile("stats.parquet").cache
>                 warning: there were 1 deprecation warning(s); re-run
>                 with -deprecation for details
>                 SLF4J: Failed to load class
>                 "org.slf4j.impl.StaticLoggerBinder".
>                 SLF4J: Defaulting to no-operation (NOP) logger
>                 implementation
>                 SLF4J: See
>                 http://www.slf4j.org/codes.html#StaticLoggerBinder for
>                 further details.
>                 15/09/07 13:45:49 INFO HiveContext: Initializing
>                 HiveMetastoreConnection version 0.13.1 using Spark
>                 classes.
>                 15/09/07 13:45:49 WARN NativeCodeLoader: Unable to
>                 load native-hadoop library for your platform... using
>                 builtin-java classes where applicable
>                 15/09/07 13:45:49 INFO HiveMetaStore: 0: Opening raw
>                 store with implemenation
>                 class:org.apache.hadoop.hive.metastore.ObjectStore
>                 15/09/07 13:45:49 INFO ObjectStore: ObjectStore,
>                 initialize called
>                 15/09/07 13:45:50 INFO Persistence: Property
>                 hive.metastore.integral.jdo.pushdown unknown - will be
>                 ignored
>                 15/09/07 13:45:50 INFO Persistence: Property
>                 datanucleus.cache.level2 unknown - will be ignored
>                 15/09/07 13:45:50 WARN Connection: BoneCP specified
>                 but not present in CLASSPATH (or one of dependencies)
>                 15/09/07 13:45:50 WARN Connection: BoneCP specified
>                 but not present in CLASSPATH (or one of dependencies)
>                 15/09/07 13:45:51 INFO ObjectStore: Setting MetaStore
>                 object pin classes with
>                 hive.metastore.cache.pinobjtypes="Table,StorageDescriptor,SerDeInfo,Partition,Database,Type,FieldSchema,Order"
>                 15/09/07 13:45:51 INFO MetaStoreDirectSql: MySQL check
>                 failed, assuming we are not on mysql: Lexical error at
>                 line 1, column 5. Encountered: "@" (64), after : "".
>                 15/09/07 13:45:52 INFO Datastore: The class
>                 "org.apache.hadoop.hive.metastore.model.MFieldSchema"
>                 is tagged as "embedded-only" so does not have its own
>                 datastore table.
>                 15/09/07 13:45:52 INFO Datastore: The class
>                 "org.apache.hadoop.hive.metastore.model.MOrder" is
>                 tagged as "embedded-only" so does not have its own
>                 datastore table.
>                 15/09/07 13:45:52 INFO Datastore: The class
>                 "org.apache.hadoop.hive.metastore.model.MFieldSchema"
>                 is tagged as "embedded-only" so does not have its own
>                 datastore table.
>                 15/09/07 13:45:52 INFO Datastore: The class
>                 "org.apache.hadoop.hive.metastore.model.MOrder" is
>                 tagged as "embedded-only" so does not have its own
>                 datastore table.
>                 15/09/07 13:45:52 INFO Query: Reading in results for
>                 query "org.datanucleus.store.rdbms.query.SQLQuery@0"
>                 since the connection used is closing
>                 15/09/07 13:45:52 INFO ObjectStore: Initialized
>                 ObjectStore
>                 15/09/07 13:45:53 INFO HiveMetaStore: Added admin role
>                 in metastore
>                 15/09/07 13:45:53 INFO HiveMetaStore: Added public
>                 role in metastore
>                 15/09/07 13:45:53 INFO HiveMetaStore: No user is added
>                 in admin role, since config is empty
>                 15/09/07 13:45:53 INFO SessionState: No Tez session
>                 required at this point. hive.execution.engine=mr.
>                 15/09/07 13:45:53 INFO MemoryStore:
>                 ensureFreeSpace(213456) called with curMem=0,
>                 maxMem=278019440
>                 15/09/07 13:45:53 INFO MemoryStore: Block broadcast_0
>                 stored as values in memory (estimated size 208.5 KB,
>                 free 264.9 MB)
>                 15/09/07 13:45:53 INFO MemoryStore:
>                 ensureFreeSpace(19750) called with curMem=213456,
>                 maxMem=278019440
>                 15/09/07 13:45:53 INFO MemoryStore: Block
>                 broadcast_0_piece0 stored as bytes in memory
>                 (estimated size 19.3 KB, free 264.9 MB)
>                 15/09/07 13:45:53 INFO BlockManagerInfo: Added
>                 broadcast_0_piece0 in memory on localhost:60973 (size:
>                 19.3 KB, free: 265.1 MB)
>                 15/09/07 13:45:53 INFO SparkContext: Created broadcast
>                 0 from cache at <console>:19
>                 dataFrame: org.apache.spark.sql.DataFrame =
>                 [date_time: string, type: string, type_value: string,
>                 newly_observed: string, risk_score: string,
>                 dates_seen: array<string>, hosts_current:
>                 array<string>, hosts_historical: array<string>,
>                 md5s_current: array<string>, md5s_historical:
>                 array<string>, processes_current: array<string>,
>                 processes_historical: array<string>, paths_current:
>                 array<string>, paths_historical: array<string>,
>                 ports_current: array<string>, ports_historical:
>                 array<string>]
>
>                 scala> val values = df.first
>                 15/09/07 13:45:58 INFO deprecation:
>                 mapred.max.split.size is deprecated. Instead, use
>                 mapreduce.input.fileinputformat.split.maxsize
>                 15/09/07 13:45:58 INFO deprecation:
>                 mapred.min.split.size is deprecated. Instead, use
>                 mapreduce.input.fileinputformat.split.minsize
>                 15/09/07 13:45:58 INFO
>                 ParquetRelation2$$anonfun$buildScan$1$$anon$1$$anon$2:
>                 Using Task Side Metadata Split Strategy
>                 15/09/07 13:45:58 INFO SparkContext: Starting job:
>                 first at <console>:21
>                 15/09/07 13:45:59 INFO DAGScheduler: Got job 0 (first
>                 at <console>:21) with 1 output partitions
>                 (allowLocal=false)
>                 15/09/07 13:45:59 INFO DAGScheduler: Final stage:
>                 ResultStage 0(first at <console>:21)
>                 15/09/07 13:45:59 INFO DAGScheduler: Parents of final
>                 stage: List()
>                 15/09/07 13:45:59 INFO DAGScheduler: Missing parents:
>                 List()
>                 15/09/07 13:45:59 INFO DAGScheduler: Submitting
>                 ResultStage 0 (MapPartitionsRDD[4] at first at
>                 <console>:21), which has no missing parents
>                 15/09/07 13:45:59 INFO MemoryStore:
>                 ensureFreeSpace(22552) called with curMem=233206,
>                 maxMem=278019440
>                 15/09/07 13:45:59 INFO MemoryStore: Block broadcast_1
>                 stored as values in memory (estimated size 22.0 KB,
>                 free 264.9 MB)
>                 15/09/07 13:45:59 INFO MemoryStore:
>                 ensureFreeSpace(8219) called with curMem=255758,
>                 maxMem=278019440
>                 15/09/07 13:45:59 INFO MemoryStore: Block
>                 broadcast_1_piece0 stored as bytes in memory
>                 (estimated size 8.0 KB, free 264.9 MB)
>                 15/09/07 13:45:59 INFO BlockManagerInfo: Added
>                 broadcast_1_piece0 in memory on localhost:60973 (size:
>                 8.0 KB, free: 265.1 MB)
>                 15/09/07 13:45:59 INFO SparkContext: Created broadcast
>                 1 from broadcast at DAGScheduler.scala:874
>                 15/09/07 13:45:59 INFO DAGScheduler: Submitting 1
>                 missing tasks from ResultStage 0 (MapPartitionsRDD[4]
>                 at first at <console>:21)
>                 15/09/07 13:45:59 INFO TaskSchedulerImpl: Adding task
>                 set 0.0 with 1 tasks
>                 15/09/07 13:45:59 INFO TaskSetManager: Starting task
>                 0.0 in stage 0.0 (TID 0, localhost, PROCESS_LOCAL,
>                 1894 bytes)
>                 15/09/07 13:45:59 INFO Executor: Running task 0.0 in
>                 stage 0.0 (TID 0)
>                 15/09/07 13:45:59 INFO CacheManager: Partition rdd_2_0
>                 not found, computing it
>                 15/09/07 13:45:59 INFO
>                 ParquetRelation2$$anonfun$buildScan$1$$anon$1: Input
>                 split: ParquetInputSplit{part:
>                 file:/home/hivedata/spark-1.4.1-bin-hadoop2.6/stats.parquet
>                 start: 0 end: 2373831 length: 2373831 hosts: []
>                 requestedSchema: message root {
>                   optional binary date_time (UTF8);
>                   optional binary type (UTF8);
>                   optional binary type_value (UTF8);
>                   optional binary newly_observed (UTF8);
>                   optional binary risk_score (UTF8);
>                   optional group dates_seen (LIST) {
>                     repeated group bag {
>                       optional binary array (UTF8);
>                     }
>                   }
>                   optional group hosts_current (LIST) {
>                     repeated group bag {
>                       optional binary array (UTF8);
>                     }
>                   }
>                   optional group hosts_historical (LIST) {
>                     repeated group bag {
>                       optional binary array (UTF8);
>                     }
>                   }
>                   optional group md5s_current (LIST) {
>                     repeated group bag {
>                       optional binary array (UTF8);
>                     }
>                   }
>                   optional group md5s_historical (LIST) {
>                     repeated group bag {
>                       optional binary array (UTF8);
>                     }
>                   }
>                   optional group processes_current (LIST) {
>                     repeated group bag {
>                       optional binary array (UTF8);
>                     }
>                   }
>                   optional group processes_historical (LIST) {
>                     repeated group bag {
>                       optional binary array (UTF8);
>                     }
>                   }
>                   optional group paths_current (LIST) {
>                     repeated group bag {
>                       optional binary array (UTF8);
>                     }
>                   }
>                   optional group paths_historical (LIST) {
>                     repeated group bag {
>                       optional binary array (UTF8);
>                     }
>                   }
>                   optional group ports_current (LIST) {
>                     repeated group bag {
>                       optional binary array (UTF8);
>                     }
>                   }
>                   optional group ports_historical (LIST) {
>                     repeated group bag {
>                       optional binary array (UTF8);
>                     }
>                   }
>                 }
>                  readSupportMetadata:
>                 {org.apache.spark.sql.parquet.row.requested_schema={"type":"struct","fields":[{"name":"date_time","type":"string","nullable":true,"metadata":{}},{"name":"type","type":"string","nullable":true,"metadata":{}},{"name":"type_value","type":"string","nullable":true,"metadata":{}},{"name":"newly_observed","type":"string","nullable":true,"metadata":{}},{"name":"risk_score","type":"string","nullable":true,"metadata":{}},{"name":"dates_seen","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"hosts_current","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"hosts_historical","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"md5s_current","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"md5s_historical","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"processes_current","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"processes_historical","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"paths_current","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"paths_historical","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"ports_current","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"ports_historical","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}}]},
>                 org.apache.spark.sql.parquet.row.metadata={"type":"struct","fields":[{"name":"date_time","type":"string","nullable":true,"metadata":{}},{"name":"type","type":"string","nullable":true,"metadata":{}},{"name":"type_value","type":"string","nullable":true,"metadata":{}},{"name":"newly_observed","type":"string","nullable":true,"metadata":{}},{"name":"risk_score","type":"string","nullable":true,"metadata":{}},{"name":"dates_seen","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"hosts_current","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"hosts_historical","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"md5s_current","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"md5s_historical","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"processes_current","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"processes_historical","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"paths_current","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"paths_historical","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"ports_current","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}},{"name":"ports_historical","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}}]}}}
>                 15/09/07 13:45:59 ERROR Executor: Exception in task
>                 0.0 in stage 0.0 (TID 0)
>                 parquet.io.ParquetDecodingException: Can not read
>                 value at 0 in block -1 in file
>                 file:/home/hivedata/spark-1.4.1-bin-hadoop2.6/stats.parquet
>                 at
>                 parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:213)
>                 at
>                 parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:204)
>                 at
>                 org.apache.spark.sql.sources.SqlNewHadoopRDD$$anon$1.hasNext(SqlNewHadoopRDD.scala:163)
>                 at
>                 org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
>                 at
>                 scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
>                 at
>                 org.apache.spark.sql.columnar.InMemoryRelation$$anonfun$3$$anon$1.hasNext(InMemoryColumnarTableScan.scala:160)
>                 at
>                 org.apache.spark.storage.MemoryStore.unrollSafely(MemoryStore.scala:276)
>                 at
>                 org.apache.spark.CacheManager.putInBlockManager(CacheManager.scala:171)
>                 at
>                 org.apache.spark.CacheManager.getOrCompute(CacheManager.scala:78)
>                 at org.apache.spark.rdd.RDD.iterator(RDD.scala:242)
>                 at
>                 org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35)
>                 at
>                 org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277)
>                 at org.apache.spark.rdd.RDD.iterator(RDD.scala:244)
>                 at
>                 org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35)
>                 at
>                 org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277)
>                 at org.apache.spark.rdd.RDD.iterator(RDD.scala:244)
>                 at
>                 org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:63)
>                 at org.apache.spark.scheduler.Task.run(Task.scala:70)
>                 at
>                 org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>                 at
>                 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>                 at
>                 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>                 at java.lang.Thread.run(Thread.java:744)
>                 Caused by: java.lang.ArrayIndexOutOfBoundsException: -1
>                 at java.util.ArrayList.elementData(ArrayList.java:403)
>                 at java.util.ArrayList.get(ArrayList.java:416)
>                 at parquet.io.GroupColumnIO.getLast(GroupColumnIO.java:95)
>                 at parquet.io.GroupColumnIO.getLast(GroupColumnIO.java:95)
>                 at parquet.io.GroupColumnIO.getLast(GroupColumnIO.java:95)
>                 at
>                 parquet.io.PrimitiveColumnIO.getLast(PrimitiveColumnIO.java:80)
>                 at
>                 parquet.io.PrimitiveColumnIO.isLast(PrimitiveColumnIO.java:74)
>                 at
>                 parquet.io.RecordReaderImplementation.<init>(RecordReaderImplementation.java:290)
>                 at
>                 parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:131)
>                 at
>                 parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:96)
>                 at
>                 parquet.filter2.compat.FilterCompat$NoOpFilter.accept(FilterCompat.java:136)
>                 at
>                 parquet.io.MessageColumnIO.getRecordReader(MessageColumnIO.java:96)
>                 at
>                 parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:126)
>                 at
>                 parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:193)
>                 ... 21 more
>                 15/09/07 13:45:59 WARN TaskSetManager: Lost task 0.0
>                 in stage 0.0 (TID 0, localhost):
>                 parquet.io.ParquetDecodingException: Can not read
>                 value at 0 in block -1 in file
>                 file:/home/hivedata/spark-1.4.1-bin-hadoop2.6/stats.parquet
>                 at
>                 parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:213)
>                 at
>                 parquet.hadoop.ParquetRecordReader.nextKeyValue(ParquetRecordReader.java:204)
>                 at
>                 org.apache.spark.sql.sources.SqlNewHadoopRDD$$anon$1.hasNext(SqlNewHadoopRDD.scala:163)
>                 at
>                 org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
>                 at
>                 scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327)
>                 at
>                 org.apache.spark.sql.columnar.InMemoryRelation$$anonfun$3$$anon$1.hasNext(InMemoryColumnarTableScan.scala:160)
>                 at
>                 org.apache.spark.storage.MemoryStore.unrollSafely(MemoryStore.scala:276)
>                 at
>                 org.apache.spark.CacheManager.putInBlockManager(CacheManager.scala:171)
>                 at
>                 org.apache.spark.CacheManager.getOrCompute(CacheManager.scala:78)
>                 at org.apache.spark.rdd.RDD.iterator(RDD.scala:242)
>                 at
>                 org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35)
>                 at
>                 org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277)
>                 at org.apache.spark.rdd.RDD.iterator(RDD.scala:244)
>                 at
>                 org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35)
>                 at
>                 org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277)
>                 at org.apache.spark.rdd.RDD.iterator(RDD.scala:244)
>                 at
>                 org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:63)
>                 at org.apache.spark.scheduler.Task.run(Task.scala:70)
>                 at
>                 org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213)
>                 at
>                 java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>                 at
>                 java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>                 at java.lang.Thread.run(Thread.java:744)
>                 Caused by: java.lang.ArrayIndexOutOfBoundsException: -1
>                 at java.util.ArrayList.elementData(ArrayList.java:403)
>                 at java.util.ArrayList.get(ArrayList.java:416)
>                 at parquet.io.GroupColumnIO.getLast(GroupColumnIO.java:95)
>                 at parquet.io.GroupColumnIO.getLast(GroupColumnIO.java:95)
>                 at parquet.io.GroupColumnIO.getLast(GroupColumnIO.java:95)
>                 at
>                 parquet.io.PrimitiveColumnIO.getLast(PrimitiveColumnIO.java:80)
>                 at
>                 parquet.io.PrimitiveColumnIO.isLast(PrimitiveColumnIO.java:74)
>                 at
>                 parquet.io.RecordReaderImplementation.<init>(RecordReaderImplementation.java:290)
>                 at
>                 parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:131)
>                 at
>                 parquet.io.MessageColumnIO$1.visit(MessageColumnIO.java:96)
>                 at
>                 parquet.filter2.compat.FilterCompat$NoOpFilter.accept(FilterCompat.java:136)
>                 at
>                 parquet.io.MessageColumnIO.getRecordReader(MessageColumnIO.java:96)
>                 at
>                 parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:126)
>                 at
>                 parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:193)
>                 ... 21 more
>                 {code}
>
>                 The file reads fine with parquet-tools.  The code has
>                 been downloaded from
>                 http://spark.apache.org/downloads.html.
>
>                 The code does works with non-array types.
>
>                 I would file a bug, but Apache JIRA refuses to
>                 register me (actually, re-register since I lost my
>                 previous account).
>
>                 -- 
>                 alexvk@gmail.com <mailto:alexvk@gmail.com>
>
>
>
>
>         -- 
>         Alex Kozlov
>         (408) 507-4987 <tel:%28408%29%20507-4987>
>         (408) 830-9982 <tel:%28408%29%20830-9982> fax
>         (650) 887-2135 <tel:%28650%29%20887-2135> efax
>         alexvk@gmail.com <mailto:alexvk@gmail.com>
>
>
>
>
>
> -- 
> Alex Kozlov
> (408) 507-4987
> (408) 830-9982 fax
> (650) 887-2135 efax
> alexvk@gmail.com <mailto:alexvk@gmail.com>


Mime
View raw message