You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@drill.apache.org by "Hanifi Gunes (JIRA)" <ji...@apache.org> on 2014/11/19 20:43:36 UTC

[jira] [Commented] (DRILL-1745) order by a json array element caused IndexOutOfBoundsException

    [ https://issues.apache.org/jira/browse/DRILL-1745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14218378#comment-14218378 ] 

Hanifi Gunes commented on DRILL-1745:
-------------------------------------

Can you upload the first 100 lines of your dataset?

> order by a json array element caused IndexOutOfBoundsException
> --------------------------------------------------------------
>
>                 Key: DRILL-1745
>                 URL: https://issues.apache.org/jira/browse/DRILL-1745
>             Project: Apache Drill
>          Issue Type: Bug
>          Components: Storage - JSON
>    Affects Versions: 0.7.0
>            Reporter: Chun Chang
>            Assignee: Hanifi Gunes
>             Fix For: 0.7.0
>
>
> #Thu Nov 13 22:54:15 EST 2014
> git.commit.id.abbrev=108d29f
> JSON data has 1 million rows. The following query works:
> 0: jdbc:drill:schema=dfs> select t.gbyt, t.id from `complex.json` t order by t.id limit 10;
> +------------+------------+
> |    gbyt    |     id     |
> +------------+------------+
> | soa        | 1          |
> | oooa       | 2          |
> | bool       | 3          |
> | nul        | 4          |
> | gbyi       | 5          |
> | bool       | 6          |
> | soa        | 7          |
> | bool       | 8          |
> | oooa       | 9          |
> | oooa       | 10         |
> +------------+------------+
> 10 rows selected (102.968 seconds)
> But if I added the following element, it will cause the exception. Without order by also works.
> 0: jdbc:drill:schema=dfs> select t.gbyt, t.id, t.ooa[0].`in` zeroin from `complex.json` t order by t.id limit 10;
> +------------+------------+------------+
> |    gbyt    |     id     |   zeroin   |
> +------------+------------+------------+
> Query failed: Failure while running fragment., writerIndex: 4098 (expected: readerIndex(0) <= writerIndex <= capacity(4096)) [ 81970fea-5f90-402d-a437-64cad4c4ebc4 on qa-node120.qa.lab:31010 ]
> java.lang.RuntimeException: java.sql.SQLException: Failure while executing query.
> 	at sqlline.SqlLine$IncrementalRows.hasNext(SqlLine.java:2514)
> 	at sqlline.SqlLine$TableOutputFormat.print(SqlLine.java:2148)
> 	at sqlline.SqlLine.print(SqlLine.java:1809)
> 	at sqlline.SqlLine$Commands.execute(SqlLine.java:3766)
> 	at sqlline.SqlLine$Commands.sql(SqlLine.java:3663)
> 	at sqlline.SqlLine.dispatch(SqlLine.java:889)
> 	at sqlline.SqlLine.begin(SqlLine.java:763)
> 	at sqlline.SqlLine.start(SqlLine.java:498)
> 	at sqlline.SqlLine.main(SqlLine.java:460)
> 0: jdbc:drill:schema=dfs>
> Here is the exception stack:
> 2014-11-18 18:42:51,791 [f72592cc-8bb3-479a-b137-adb82787f44e:frag:2:0] ERROR o.a.drill.exec.ops.FragmentContext - Fragment Context received failure.
> java.lang.IndexOutOfBoundsException: writerIndex: 4098 (expected: readerIndex(0) <= writerIndex <= capacity(4096))
> 	at io.netty.buffer.AbstractByteBuf.writerIndex(AbstractByteBuf.java:88) ~[netty-buffer-4.0.24.Final.jar:4.0.24.Final]
> 	at org.apache.drill.exec.vector.VectorTrimmer.trim(VectorTrimmer.java:27) ~[drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.vector.UInt1Vector$Mutator.setValueCount(UInt1Vector.java:420) ~[drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.vector.NullableVarCharVector$Mutator.setValueCount(NullableVarCharVector.java:547) ~[drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.vector.complex.MapVector$Mutator.setValueCount(MapVector.java:420) ~[drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.vector.complex.MapVector$Mutator.setValueCount(MapVector.java:420) ~[drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.vector.complex.RepeatedMapVector$Mutator.setValueCount(RepeatedMapVector.java:615) ~[drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.vector.complex.MapVector$Mutator.setValueCount(MapVector.java:420) ~[drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.vector.complex.impl.SingleMapWriter.setValueCount(SingleMapWriter.java:163) ~[drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.vector.complex.impl.VectorContainerWriter.setValueCount(VectorContainerWriter.java:73) ~[drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.store.easy.json.JSONRecordReader2.next(JSONRecordReader2.java:133) ~[drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.physical.impl.ScanBatch.next(ScanBatch.java:191) ~[drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:124) [drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:86) [drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:76) [drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.record.AbstractSingleRecordBatch.innerNext(AbstractSingleRecordBatch.java:52) [drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.physical.impl.project.ProjectRecordBatch.innerNext(ProjectRecordBatch.java:129) [drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.record.AbstractRecordBatch.next(AbstractRecordBatch.java:106) [drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.physical.impl.validate.IteratorValidatorBatchIterator.next(IteratorValidatorBatchIterator.java:124) [drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.physical.impl.BaseRootExec.next(BaseRootExec.java:67) [drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.physical.impl.partitionsender.PartitionSenderRootExec.innerNext(PartitionSenderRootExec.java:141) [drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.physical.impl.BaseRootExec.next(BaseRootExec.java:57) [drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.work.fragment.FragmentExecutor.run(FragmentExecutor.java:113) [drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at org.apache.drill.exec.work.WorkManager$RunnableWrapper.run(WorkManager.java:249) [drill-java-exec-0.6.0.r2-incubating-SNAPSHOT-rebuffed.jar:0.6.0.r2-incubating-SNAPSHOT]
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) [na:1.7.0_45]
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) [na:1.7.0_45]
> 	at java.lang.Thread.run(Thread.java:744) [na:1.7.0_45]
> Without order by works:
> 0: jdbc:drill:schema=dfs> select t.gbyt, t.id, t.ooa[0].`in` zeroin from `complex.json` t limit 10;
> +------------+------------+------------+
> |    gbyt    |     id     |   zeroin   |
> +------------+------------+------------+
> | soa        | 1          | 1          |
> | oooa       | 2          | 2          |
> | bool       | 3          | 3          |
> | nul        | 4          | 4          |
> | gbyi       | 5          | 5          |
> | bool       | 6          | null       |
> | soa        | 7          | 7          |
> | bool       | 8          | 8          |
> | oooa       | 9          | 9          |
> | oooa       | 10         | 10         |
> +------------+------------+------------+
> 10 rows selected (0.255 seconds)
> Plan:
> 0: jdbc:drill:schema=dfs> explain plan for select t.gbyt, t.id, t.ooa[0].`in` zeroin from `complex.json` t order by t.id limit 10;
> +------------+------------+
> |    text    |    json    |
> +------------+------------+
> | 00-00    Screen
> 00-01      Project(gbyt=[$0], id=[$1], zeroin=[$2])
> 00-02        SelectionVectorRemover
> 00-03          Limit(fetch=[10])
> 00-04            SingleMergeExchange(sort0=[1 ASC])
> 01-01              SelectionVectorRemover
> 01-02                TopN(limit=[10])
> 01-03                  HashToRandomExchange(dist0=[[$1]])
> 02-01                    Project(gbyt=[$1], id=[$2], zeroin=[ITEM(ITEM($0, 0), 'in')])
> 02-02                      Scan(groupscan=[EasyGroupScan [selectionRoot=/drill/testdata/mondrian/complex.json, numFiles=1, columns=[`gbyt`, `id`, `ooa`[0].`in`], files=[maprfs:/drill/testdata/mondrian/complex.json]]])
>  | {
>   "head" : {
>     "version" : 1,
>     "generator" : {
>       "type" : "ExplainHandler",
>       "info" : ""
>     },
>     "type" : "APACHE_DRILL_PHYSICAL",
>     "options" : [ ],
>     "queue" : 0,
>     "resultMode" : "EXEC"
>   },
>   "graph" : [ {
>     "pop" : "fs-scan",
>     "@id" : 131074,
>     "files" : [ "maprfs:/drill/testdata/mondrian/complex.json" ],
>     "storage" : {
>       "type" : "file",
>       "enabled" : true,
>       "connection" : "maprfs:///",
>       "workspaces" : {
>         "root" : {
>           "location" : "/",
>           "writable" : false,
>           "defaultInputFormat" : null
>         },
>         "tmp" : {
>           "location" : "/tmp",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "default" : {
>           "location" : "/drill/testdata/mondrian",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDir" : {
>           "location" : "/drill/testdata/",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirAmplab" : {
>           "location" : "/drill/testdata/amplab",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirInformationSchema" : {
>           "location" : "/drill/testdata/information-schema",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirUdfs" : {
>           "location" : "/drill/testdata/udfs/",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirP1" : {
>           "location" : "/drill/testdata/p1tests",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "Join" : {
>           "location" : "/drill/testdata/join",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirExchanges" : {
>           "location" : "/drill/testdata/exchanges_test",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "TpcHMulti" : {
>           "location" : "/drill/testdata/tpch-multi",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "TpcHMulti100" : {
>           "location" : "/drill/testdata/SF100",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "TpcHMulti1" : {
>           "location" : "/drill/testdata/tpch_SF1",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirExplicit" : {
>           "location" : "/drill/testdata/explicit_cast",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirImplicit" : {
>           "location" : "/drill/testdata/implicit_cast",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirImplicit1" : {
>           "location" : "/drill/testdata/implicit_cast",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirTPCDS" : {
>           "location" : "/user/root/tpcds/parquet",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "TPCDS" : {
>           "location" : "/drill/testdata/tpcds",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillMondrian" : {
>           "location" : "/user/root/mondrian",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirDatetime" : {
>           "location" : "/drill/testdata/datetime/datasources",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirViews" : {
>           "location" : "/drill/testdata/views/",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirNumerical" : {
>           "location" : "/drill/testdata/numerical/",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirJson" : {
>           "location" : "/drill/testdata/json_storage/",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirTestNewWS" : {
>           "location" : "/drill/testdata/newWS/",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirTpch01Text" : {
>           "location" : "/drill/testdata/Tpch0.01/text/",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirTpch01Json" : {
>           "location" : "/drill/testdata/Tpch0.01/json/",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirTpch01Parquet" : {
>           "location" : "/drill/testdata/Tpch0.01/parquet/",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirConvert" : {
>           "location" : "/drill/testdata/convert",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirTpch100Text" : {
>           "location" : "/drill/testdata/tpch100/text/",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirTpch100Parquet" : {
>           "location" : "/drill/testdata/tpch100/parquet",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirAggregate1parquet" : {
>           "location" : "/drill/testdata/tpcds/parquet/s1",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirAggregate1csv" : {
>           "location" : "/drill/testdata/tpcds/csv/s1",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirAggregate1json" : {
>           "location" : "/drill/testdata/tpcds/json/s1",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirMondrian" : {
>           "location" : "/drill/testdata/mondrian",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "drillTestDirTpcdsImpalaSF1" : {
>           "location" : "/drill/testdata/tpcds-impala-sf1",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "sandbox" : {
>           "location" : "/sandbox",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "sandbox-logs" : {
>           "location" : "/sandbox/flat",
>           "writable" : true,
>           "defaultInputFormat" : null
>         },
>         "sandbox-json" : {
>           "location" : "/sandbox/json",
>           "writable" : true,
>           "defaultInputFormat" : null
>         }
>       },
>       "formats" : {
>         "psv" : {
>           "type" : "text",
>           "extensions" : [ "tbl" ],
>           "delimiter" : "|"
>         },
>         "dsv" : {
>           "type" : "text",
>           "extensions" : [ "dat" ],
>           "delimiter" : "|"
>         },
>         "csv" : {
>           "type" : "text",
>           "extensions" : [ "csv" ],
>           "delimiter" : ","
>         },
>         "tsv" : {
>           "type" : "text",
>           "extensions" : [ "tsv" ],
>           "delimiter" : "\t"
>         },
>         "parquet" : {
>           "type" : "parquet"
>         },
>         "json" : {
>           "type" : "json"
>         }
>       }
>     },
>     "format" : {
>       "type" : "json"
>     },
>     "columns" : [ "`gbyt`", "`id`", "`ooa`[0].`in`" ],
>     "selectionRoot" : "/drill/testdata/mondrian/complex.json",
>     "cost" : 1186767.0
>   }, {
>     "pop" : "project",
>     "@id" : 131073,
>     "exprs" : [ {
>       "ref" : "`gbyt`",
>       "expr" : "`gbyt`"
>     }, {
>       "ref" : "`id`",
>       "expr" : "`id`"
>     }, {
>       "ref" : "`zeroin`",
>       "expr" : "`ooa`[0].`in`"
>     } ],
>     "child" : 131074,
>     "initialAllocation" : 1000000,
>     "maxAllocation" : 10000000000,
>     "cost" : 1186767.0
>   }, {
>     "pop" : "hash-to-random-exchange",
>     "@id" : 65539,
>     "child" : 131073,
>     "expr" : "hash(`id`) ",
>     "initialAllocation" : 1000000,
>     "maxAllocation" : 10000000000,
>     "cost" : 1186767.0
>   }, {
>     "pop" : "top-n",
>     "@id" : 65538,
>     "child" : 65539,
>     "orderings" : [ {
>       "order" : "ASC",
>       "expr" : "`id`",
>       "nullDirection" : "UNSPECIFIED"
>     } ],
>     "reverse" : false,
>     "limit" : 10,
>     "initialAllocation" : 1000000,
>     "maxAllocation" : 10000000000,
>     "cost" : 1186767.0
>   }, {
>     "pop" : "selection-vector-remover",
>     "@id" : 65537,
>     "child" : 65538,
>     "initialAllocation" : 1000000,
>     "maxAllocation" : 10000000000,
>     "cost" : 1186767.0
>   }, {
>     "pop" : "single-merge-exchange",
>     "@id" : 4,
>     "child" : 65537,
>     "orderings" : [ {
>       "order" : "ASC",
>       "expr" : "`id`",
>       "nullDirection" : "UNSPECIFIED"
>     } ],
>     "initialAllocation" : 1000000,
>     "maxAllocation" : 10000000000,
>     "cost" : 1186767.0
>   }, {
>     "pop" : "limit",
>     "@id" : 3,
>     "child" : 4,
>     "first" : 0,
>     "last" : 10,
>     "initialAllocation" : 1000000,
>     "maxAllocation" : 10000000000,
>     "cost" : 1186767.0
>   }, {
>     "pop" : "selection-vector-remover",
>     "@id" : 2,
>     "child" : 3,
>     "initialAllocation" : 1000000,
>     "maxAllocation" : 10000000000,
>     "cost" : 1186767.0
>   }, {
>     "pop" : "project",
>     "@id" : 1,
>     "exprs" : [ {
>       "ref" : "`gbyt`",
>       "expr" : "`gbyt`"
>     }, {
>       "ref" : "`id`",
>       "expr" : "`id`"
>     }, {
>       "ref" : "`zeroin`",
>       "expr" : "`zeroin`"
>     } ],
>     "chil |
> +------------+------------+
> 1 row selected (0.096 seconds)



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)