You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by "Jeffrey Olchovy (JIRA)" <ji...@apache.org> on 2016/06/12 22:40:21 UTC
[jira] [Created] (PARQUET-631) ScroogeWriteSupport can not handle
structs with union members
Jeffrey Olchovy created PARQUET-631:
---------------------------------------
Summary: ScroogeWriteSupport can not handle structs with union members
Key: PARQUET-631
URL: https://issues.apache.org/jira/browse/PARQUET-631
Project: Parquet
Issue Type: Bug
Components: parquet-mr
Affects Versions: 1.8.0
Reporter: Jeffrey Olchovy
When attempting to write ThriftStruct instances that contain union members, exceptions are encountered:
1. When writing a struct that consists solely of a union member, {{org.apache.parquet.thrift.projection.ThriftProjectionException: No columns have been selected}} is encountered
2. When writing a struct that consists of a union member plus other types of members, {{org.apache.parquet.io.InvalidRecordException: could not get child...}} is encountered
Given the following Thrift definitions:
{noformat}
namespace java io.narrative.dto
union SimpleUnion {
1: i64 i,
2: string s
}
struct SimpleDto {
1: required i64 i
}
struct NestedDto {
1: required SimpleDto simpleDto
}
struct SimpleUnionDto {
1: required SimpleUnion simpleUnion
}
union ComplexUnion {
1: SimpleDto maybeSimpleDto
2: NestedDto maybeNestedDto
}
struct NestedWithSimpleUnionDto {
1: required SimpleDto simpleDto
2: required SimpleUnion simpleUnion
}
struct NestedWithComplexUnionDto {
1: required SimpleDto simpleDto
2: required ComplexUnion complexUnion
}
struct NestedWithContainerOfSimpleUnionDto {
1: required SimpleDto simpleDto
2: required list<SimpleUnion> simpleUnions
}
struct NestedWithContainerOfComplexUnionDto {
1: required SimpleDto simpleDto
2: required list<ComplexUnion> complexUnions
}
{noformat}
And the following scalatest specifications:
{noformat}
import java.io.File
import com.twitter.scrooge.ThriftStruct
import org.apache.hadoop.fs.Path
import org.apache.hadoop.conf.Configuration
import org.apache.parquet.hadoop.ParquetWriter
import org.apache.parquet.hadoop.metadata.CompressionCodecName
import org.apache.parquet.scrooge.ScroogeWriteSupport
import org.scalatest.FlatSpec
import io.narrative.dto._
class ScroogeParquetWriterSpec extends FlatSpec {
import ParquetWriter._
def write[A <: ThriftStruct : Manifest](a: A): Unit = {
val file = File.createTempFile("tmp", ".snappy.parquet")
file.delete()
val path = new Path(file.toURI)
val writeSupport = new ScroogeWriteSupport[A](manifest[A].runtimeClass.asInstanceOf[Class[A]])
val underlying = new ParquetWriter[A](path, writeSupport, CompressionCodecName.SNAPPY, DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE)
try {
underlying.write(a)
} finally {
underlying.close()
}
}
behavior of "ParquetWriter"
it should "write Scrooge records with simple members" in {
val simpleDto = SimpleDto(42)
write(simpleDto)
}
it should "write Scrooge records with struct members" in {
val simpleDto = SimpleDto(42)
val nestedDto = NestedDto(simpleDto)
write(nestedDto)
}
it should "write Scrooge records with a union member that contains simple types" in {
val simpleUnionDto = SimpleUnionDto(SimpleUnion.I(42))
write(simpleUnionDto)
}
it should "write Scrooge records with struct and union members that contain simple types" in {
val simpleDto = SimpleDto(42)
val nestedWithSimpleUnion = NestedWithSimpleUnionDto(simpleDto, SimpleUnion.I(42))
write(nestedWithSimpleUnion)
}
it should "write Scrooge records with struct and union members that contain struct types" in {
val simpleDto = SimpleDto(42)
val nestedWithComplexUnion = NestedWithComplexUnionDto(simpleDto, ComplexUnion.MaybeSimpleDto(simpleDto))
write(nestedWithComplexUnion)
}
it should "write Scrooge records with a struct member and a list of union members that contain simple types" in {
val simpleDto = SimpleDto(42)
val nestedWithSimpleUnions = NestedWithContainerOfSimpleUnionDto(simpleDto, Seq(SimpleUnion.I(42)))
write(nestedWithSimpleUnions)
}
it should "write Scrooge records with a struct member and a list of union members that contain struct types" in {
val simpleDto = SimpleDto(42)
val nestedWithComplexUnions = NestedWithContainerOfComplexUnionDto(simpleDto, Seq(ComplexUnion.MaybeSimpleDto(simpleDto)))
write(nestedWithComplexUnions)
}
}
{noformat}
The following output is produced:
{noformat}
[info] ScroogeParquetWriterSpec:
[info] ParquetWriter
[info] - should write Scrooge records with simple members
[info] - should write Scrooge records with struct members
[info] - should write Scrooge records with a union member that contains simple types *** FAILED ***
[info] org.apache.parquet.thrift.projection.ThriftProjectionException: No columns have been selected
[info] at org.apache.parquet.thrift.ThriftSchemaConvertVisitor.convert(ThriftSchemaConvertVisitor.java:91)
[info] at org.apache.parquet.thrift.ThriftSchemaConverter.convert(ThriftSchemaConverter.java:60)
[info] at org.apache.parquet.hadoop.thrift.AbstractThriftWriteSupport.init(AbstractThriftWriteSupport.java:88)
[info] at org.apache.parquet.hadoop.thrift.AbstractThriftWriteSupport.<init>(AbstractThriftWriteSupport.java:80)
[info] at org.apache.parquet.scrooge.ScroogeWriteSupport.<init>(ScroogeWriteSupport.java:48)
[info] at io.narrative.etl.ScroogeParquetWriterSpec.write(ThriftStructWriterSpec.scala:21)
[info] at io.narrative.etl.ScroogeParquetWriterSpec$$anonfun$3.apply$mcV$sp(ThriftStructWriterSpec.scala:47)
[info] at io.narrative.etl.ScroogeParquetWriterSpec$$anonfun$3.apply(ThriftStructWriterSpec.scala:45)
[info] at io.narrative.etl.ScroogeParquetWriterSpec$$anonfun$3.apply(ThriftStructWriterSpec.scala:45)
[info] at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
[info] ...
[info] - should write Scrooge records with struct and union members that contain simple types *** FAILED ***
[info] org.apache.parquet.io.InvalidRecordException: could not get child 1 from [GroupColumnIO simpleDto r:0 d:0 [simpleDto]]
[info] at org.apache.parquet.io.GroupColumnIO.getChild(GroupColumnIO.java:113)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$StructWriteProtocol.<init>(ParquetWriteProtocol.java:322)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$MessageWriteProtocol.<init>(ParquetWriteProtocol.java:397)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol.<init>(ParquetWriteProtocol.java:431)
[info] at org.apache.parquet.hadoop.thrift.AbstractThriftWriteSupport.prepareForWrite(AbstractThriftWriteSupport.java:121)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.initStore(InternalParquetRecordWriter.java:109)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.<init>(InternalParquetRecordWriter.java:99)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:272)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:217)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:183)
[info] ...
[info] Cause: java.lang.IndexOutOfBoundsException: Index: 1, Size: 1
[info] at java.util.ArrayList.rangeCheck(ArrayList.java:653)
[info] at java.util.ArrayList.get(ArrayList.java:429)
[info] at org.apache.parquet.io.GroupColumnIO.getChild(GroupColumnIO.java:111)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$StructWriteProtocol.<init>(ParquetWriteProtocol.java:322)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$MessageWriteProtocol.<init>(ParquetWriteProtocol.java:397)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol.<init>(ParquetWriteProtocol.java:431)
[info] at org.apache.parquet.hadoop.thrift.AbstractThriftWriteSupport.prepareForWrite(AbstractThriftWriteSupport.java:121)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.initStore(InternalParquetRecordWriter.java:109)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.<init>(InternalParquetRecordWriter.java:99)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:272)
[info] ...
[info] - should write Scrooge records with struct and union members that contain struct types *** FAILED ***
[info] org.apache.parquet.io.InvalidRecordException: could not get child 1 from [GroupColumnIO simpleDto r:0 d:0 [simpleDto]]
[info] at org.apache.parquet.io.GroupColumnIO.getChild(GroupColumnIO.java:113)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$StructWriteProtocol.<init>(ParquetWriteProtocol.java:322)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$MessageWriteProtocol.<init>(ParquetWriteProtocol.java:397)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol.<init>(ParquetWriteProtocol.java:431)
[info] at org.apache.parquet.hadoop.thrift.AbstractThriftWriteSupport.prepareForWrite(AbstractThriftWriteSupport.java:121)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.initStore(InternalParquetRecordWriter.java:109)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.<init>(InternalParquetRecordWriter.java:99)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:272)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:217)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:183)
[info] ...
[info] Cause: java.lang.IndexOutOfBoundsException: Index: 1, Size: 1
[info] at java.util.ArrayList.rangeCheck(ArrayList.java:653)
[info] at java.util.ArrayList.get(ArrayList.java:429)
[info] at org.apache.parquet.io.GroupColumnIO.getChild(GroupColumnIO.java:111)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$StructWriteProtocol.<init>(ParquetWriteProtocol.java:322)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$MessageWriteProtocol.<init>(ParquetWriteProtocol.java:397)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol.<init>(ParquetWriteProtocol.java:431)
[info] at org.apache.parquet.hadoop.thrift.AbstractThriftWriteSupport.prepareForWrite(AbstractThriftWriteSupport.java:121)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.initStore(InternalParquetRecordWriter.java:109)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.<init>(InternalParquetRecordWriter.java:99)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:272)
[info] ...
[info] - should write Scrooge records with a struct member and a list of union members that contain simple types *** FAILED ***
[info] org.apache.parquet.io.InvalidRecordException: could not get child 1 from [GroupColumnIO simpleDto r:0 d:0 [simpleDto]]
[info] at org.apache.parquet.io.GroupColumnIO.getChild(GroupColumnIO.java:113)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$StructWriteProtocol.<init>(ParquetWriteProtocol.java:322)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$MessageWriteProtocol.<init>(ParquetWriteProtocol.java:397)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol.<init>(ParquetWriteProtocol.java:431)
[info] at org.apache.parquet.hadoop.thrift.AbstractThriftWriteSupport.prepareForWrite(AbstractThriftWriteSupport.java:121)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.initStore(InternalParquetRecordWriter.java:109)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.<init>(InternalParquetRecordWriter.java:99)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:272)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:217)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:183)
[info] ...
[info] Cause: java.lang.IndexOutOfBoundsException: Index: 1, Size: 1
[info] at java.util.ArrayList.rangeCheck(ArrayList.java:653)
[info] at java.util.ArrayList.get(ArrayList.java:429)
[info] at org.apache.parquet.io.GroupColumnIO.getChild(GroupColumnIO.java:111)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$StructWriteProtocol.<init>(ParquetWriteProtocol.java:322)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$MessageWriteProtocol.<init>(ParquetWriteProtocol.java:397)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol.<init>(ParquetWriteProtocol.java:431)
[info] at org.apache.parquet.hadoop.thrift.AbstractThriftWriteSupport.prepareForWrite(AbstractThriftWriteSupport.java:121)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.initStore(InternalParquetRecordWriter.java:109)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.<init>(InternalParquetRecordWriter.java:99)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:272)
[info] ...
[info] - should write Scrooge records with a struct member and a list of union members that contain struct types *** FAILED ***
[info] org.apache.parquet.io.InvalidRecordException: could not get child 1 from [GroupColumnIO simpleDto r:0 d:0 [simpleDto]]
[info] at org.apache.parquet.io.GroupColumnIO.getChild(GroupColumnIO.java:113)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$StructWriteProtocol.<init>(ParquetWriteProtocol.java:322)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$MessageWriteProtocol.<init>(ParquetWriteProtocol.java:397)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol.<init>(ParquetWriteProtocol.java:431)
[info] at org.apache.parquet.hadoop.thrift.AbstractThriftWriteSupport.prepareForWrite(AbstractThriftWriteSupport.java:121)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.initStore(InternalParquetRecordWriter.java:109)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.<init>(InternalParquetRecordWriter.java:99)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:272)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:217)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:183)
[info] ...
[info] Cause: java.lang.IndexOutOfBoundsException: Index: 1, Size: 1
[info] at java.util.ArrayList.rangeCheck(ArrayList.java:653)
[info] at java.util.ArrayList.get(ArrayList.java:429)
[info] at org.apache.parquet.io.GroupColumnIO.getChild(GroupColumnIO.java:111)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$StructWriteProtocol.<init>(ParquetWriteProtocol.java:322)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol$MessageWriteProtocol.<init>(ParquetWriteProtocol.java:397)
[info] at org.apache.parquet.thrift.ParquetWriteProtocol.<init>(ParquetWriteProtocol.java:431)
[info] at org.apache.parquet.hadoop.thrift.AbstractThriftWriteSupport.prepareForWrite(AbstractThriftWriteSupport.java:121)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.initStore(InternalParquetRecordWriter.java:109)
[info] at org.apache.parquet.hadoop.InternalParquetRecordWriter.<init>(InternalParquetRecordWriter.java:99)
[info] at org.apache.parquet.hadoop.ParquetWriter.<init>(ParquetWriter.java:272)
[info] ...
[info] Run completed in 1 second, 818 milliseconds.
[info] Total number of tests run: 7
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 2, failed 5, canceled 0, ignored 0, pending 0
[info] *** 5 TESTS FAILED ***
{noformat}
Every specification that contained a struct with a union member fails.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)