Uploaded image for project: 'Apache AsterixDB'
  1. Apache AsterixDB
  2. ASTERIXDB-1159

Sporadic test failures in nested-open-index test

    XMLWordPrintableJSON

Details

    Description

      The test case is
      src/test/resources/runtimets/queries/nested-open-index/index-join/ngram-edit-distance/ngram-edit-distance.3.ddl.aql, which is to create an enforced index.

      org.apache.hyracks.api.exceptions.HyracksDataException: org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: java.lang.IllegalStateException: java.lang.ArrayIndexOutOfBoundsException: 16777216
      at org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory$1.produceTuple(AssignRuntimeFactory.java:150)
      at org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory$1.nextFrame(AssignRuntimeFactory.java:125)
      at org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor$2.nextFrame(AlgebricksMetaOperatorDescriptor.java:145)
      at org.apache.hyracks.dataflow.common.comm.io.AbstractFrameAppender.flush(AbstractFrameAppender.java:83)
      at org.apache.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable.close(IndexSearchOperatorNodePushable.java:196)
      at org.apache.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorNodePushable.initialize(ConstantTupleSourceOperatorNodePushable.java:54)
      at org.apache.hyracks.api.rewriter.runtime.SuperActivityOperatorNodePushable.initialize(SuperActivityOperatorNodePushable.java:85)
      at org.apache.hyracks.control.nc.Task.run(Task.java:255)
      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:745)
      Caused by: org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: java.lang.IllegalStateException: java.lang.ArrayIndexOutOfBoundsException: 16777216
      at org.apache.asterix.runtime.evaluators.functions.CastRecordDescriptor$2$1.evaluate(CastRecordDescriptor.java:95)
      at org.apache.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter$ScalarEvaluatorFactoryAdapter$1.evaluate(LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.java:110)
      at org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory$1.produceTuple(AssignRuntimeFactory.java:148)
      ... 10 more
      Caused by: java.lang.IllegalStateException: java.lang.ArrayIndexOutOfBoundsException: 16777216
      at org.apache.asterix.om.pointables.ARecordVisitablePointable.set(ARecordVisitablePointable.java:266)
      at org.apache.asterix.om.pointables.AbstractVisitablePointable.set(AbstractVisitablePointable.java:59)
      at org.apache.asterix.runtime.evaluators.functions.CastRecordDescriptor$2$1.evaluate(CastRecordDescriptor.java:90)
      ... 12 more
      Caused by: java.lang.ArrayIndexOutOfBoundsException: 16777216
      at org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer.getInt(AInt32SerializerDeserializer.java:50)
      at org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer.getRecordLength(ARecordSerializerDeserializer.java:236)
      at org.apache.asterix.om.util.NonTaggedFormatUtil.getFieldValueLength(NonTaggedFormatUtil.java:165)
      at org.apache.asterix.om.pointables.ARecordVisitablePointable.set(ARecordVisitablePointable.java:219)
      ... 14 more
      org.apache.hyracks.api.exceptions.HyracksException: Job failed on account of:
      org.apache.hyracks.api.exceptions.HyracksDataException: org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: java.lang.IllegalStateException: java.lang.ArrayIndexOutOfBoundsException: 16777216

      at org.apache.hyracks.control.cc.job.JobRun.waitForCompletion(JobRun.java:211)
      at org.apache.hyracks.control.cc.work.WaitForJobCompletionWork$1.run(WaitForJobCompletionWork.java:48)
      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:745)
      Caused by: org.apache.hyracks.api.exceptions.HyracksDataException: org.apache.hyracks.api.exceptions.HyracksDataException: org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: java.lang.IllegalStateException: java.lang.ArrayIndexOutOfBoundsException: 16777216
      at org.apache.hyracks.control.common.utils.ExceptionUtils.setNodeIds(ExceptionUtils.java:45)
      at org.apache.hyracks.control.nc.Task.run(Task.java:312)
      ... 3 more
      Caused by: org.apache.hyracks.api.exceptions.HyracksDataException: org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: java.lang.IllegalStateException: java.lang.ArrayIndexOutOfBoundsException: 16777216
      at org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory$1.produceTuple(AssignRuntimeFactory.java:150)
      at org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory$1.nextFrame(AssignRuntimeFactory.java:125)
      at org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor$2.nextFrame(AlgebricksMetaOperatorDescriptor.java:145)
      at org.apache.hyracks.dataflow.common.comm.io.AbstractFrameAppender.flush(AbstractFrameAppender.java:83)
      at org.apache.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable.close(IndexSearchOperatorNodePushable.java:196)
      at org.apache.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorNodePushable.initialize(ConstantTupleSourceOperatorNodePushable.java:54)
      at org.apache.hyracks.api.rewriter.runtime.SuperActivityOperatorNodePushable.initialize(SuperActivityOperatorNodePushable.java:85)
      at org.apache.hyracks.control.nc.Task.run(Task.java:255)
      ... 3 more
      Caused by: org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: java.lang.IllegalStateException: java.lang.ArrayIndexOutOfBoundsException: 16777216
      at org.apache.asterix.runtime.evaluators.functions.CastRecordDescriptor$2$1.evaluate(CastRecordDescriptor.java:95)
      at org.apache.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter$ScalarEvaluatorFactoryAdapter$1.evaluate(LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.java:110)
      at org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory$1.produceTuple(AssignRuntimeFactory.java:148)
      ... 10 more
      Caused by: java.lang.IllegalStateException: java.lang.ArrayIndexOutOfBoundsException: 16777216
      at org.apache.asterix.om.pointables.ARecordVisitablePointable.set(ARecordVisitablePointable.java:266)
      at org.apache.asterix.om.pointables.AbstractVisitablePointable.set(AbstractVisitablePointable.java:59)
      at org.apache.asterix.runtime.evaluators.functions.CastRecordDescriptor$2$1.evaluate(CastRecordDescriptor.java:90)
      ... 12 more
      Caused by: java.lang.ArrayIndexOutOfBoundsException: 16777216
      at org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer.getInt(AInt32SerializerDeserializer.java:50)
      at org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer.getRecordLength(ARecordSerializerDeserializer.java:236)
      at org.apache.asterix.om.util.NonTaggedFormatUtil.getFieldValueLength(NonTaggedFormatUtil.java:165)
      at org.apache.asterix.om.pointables.ARecordVisitablePointable.set(ARecordVisitablePointable.java:219)
      ... 14 more
      testFile src/test/resources/runtimets/queries/nested-open-index/index-join/ngram-edit-distance/ngram-edit-distance.3.ddl.aql raised an exception:
      java.lang.Exception: HTTP operation failed: 99
      STATUS LINE: HTTP/1.1 500 Server Error
      SUMMARY: Job failed on account of:
      org.apache.hyracks.api.exceptions.HyracksDataException: org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: java.lang.IllegalStateException: java.lang.ArrayIndexOutOfBoundsException: 16777216

      caused by: org.apache.hyracks.control.cc.job.JobRun.waitForCompletion(JobRun.java:211)
      caused by: org.apache.hyracks.control.common.utils.ExceptionUtils.setNodeIds(ExceptionUtils.java:45)
      caused by: org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory$1.produceTuple(AssignRuntimeFactory.java:150)
      caused by: org.apache.asterix.runtime.evaluators.functions.CastRecordDescriptor$2$1.evaluate(CastRecordDescriptor.java:95)
      caused by: org.apache.asterix.om.pointables.ARecordVisitablePointable.set(ARecordVisitablePointable.java:266)
      caused by: org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer.getInt(AInt32SerializerDeserializer.java:50)
      STACKTRACE: org.apache.hyracks.api.exceptions.HyracksException: Job failed on account of:
      org.apache.hyracks.api.exceptions.HyracksDataException: org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: java.lang.IllegalStateException: java.lang.ArrayIndexOutOfBoundsException: 16777216

      at org.apache.hyracks.control.cc.job.JobRun.waitForCompletion(JobRun.java:211)
      at org.apache.hyracks.control.cc.work.WaitForJobCompletionWork$1.run(WaitForJobCompletionWork.java:48)
      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:745)
      Caused by: org.apache.hyracks.api.exceptions.HyracksDataException: org.apache.hyracks.api.exceptions.HyracksDataException: org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: java.lang.IllegalStateException: java.lang.ArrayIndexOutOfBoundsException: 16777216
      at org.apache.hyracks.control.common.utils.ExceptionUtils.setNodeIds(ExceptionUtils.java:45)
      at org.apache.hyracks.control.nc.Task.run(Task.java:312)
      ... 3 more
      Caused by: org.apache.hyracks.api.exceptions.HyracksDataException: org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: java.lang.IllegalStateException: java.lang.ArrayIndexOutOfBoundsException: 16777216
      at org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory$1.produceTuple(AssignRuntimeFactory.java:150)
      at org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory$1.nextFrame(AssignRuntimeFactory.java:125)
      at org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor$2.nextFrame(AlgebricksMetaOperatorDescriptor.java:145)
      at org.apache.hyracks.dataflow.common.comm.io.AbstractFrameAppender.flush(AbstractFrameAppender.java:83)
      at org.apache.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable.close(IndexSearchOperatorNodePushable.java:196)
      at org.apache.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorNodePushable.initialize(ConstantTupleSourceOperatorNodePushable.java:54)
      at org.apache.hyracks.api.rewriter.runtime.SuperActivityOperatorNodePushable.initialize(SuperActivityOperatorNodePushable.java:85)
      at org.apache.hyracks.control.nc.Task.run(Task.java:255)
      ... 3 more
      Caused by: org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: java.lang.IllegalStateException: java.lang.ArrayIndexOutOfBoundsException: 16777216
      at org.apache.asterix.runtime.evaluators.functions.CastRecordDescriptor$2$1.evaluate(CastRecordDescriptor.java:95)
      at org.apache.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter$ScalarEvaluatorFactoryAdapter$1.evaluate(LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.java:110)
      at org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory$1.produceTuple(AssignRuntimeFactory.java:148)
      ... 10 more
      Caused by: java.lang.IllegalStateException: java.lang.ArrayIndexOutOfBoundsException: 16777216
      at org.apache.asterix.om.pointables.ARecordVisitablePointable.set(ARecordVisitablePointable.java:266)
      at org.apache.asterix.om.pointables.AbstractVisitablePointable.set(AbstractVisitablePointable.java:59)
      at org.apache.asterix.runtime.evaluators.functions.CastRecordDescriptor$2$1.evaluate(CastRecordDescriptor.java:90)
      ... 12 more
      Caused by: java.lang.ArrayIndexOutOfBoundsException: 16777216
      at org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer.getInt(AInt32SerializerDeserializer.java:50)
      at org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer.getRecordLength(ARecordSerializerDeserializer.java:236)
      at org.apache.asterix.om.util.NonTaggedFormatUtil.getFieldValueLength(NonTaggedFormatUtil.java:165)
      at org.apache.asterix.om.pointables.ARecordVisitablePointable.set(ARecordVisitablePointable.java:219)
      ... 14 more

      at org.apache.asterix.test.aql.TestExecutor.executeHttpMethod(TestExecutor.java:218)
      at org.apache.asterix.test.aql.TestExecutor.executeDDL(TestExecutor.java:312)
      at org.apache.asterix.test.aql.TestExecutor.executeTest(TestExecutor.java:411)
      at org.apache.asterix.test.runtime.ExecutionTest.test(ExecutionTest.java:170)
      at sun.reflect.GeneratedMethodAccessor28.invoke(Unknown Source)
      at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      at java.lang.reflect.Method.invoke(Method.java:497)
      at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
      at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
      at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
      at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
      at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:271)
      at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:70)
      at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:50)
      at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238)
      at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:63)
      at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:236)
      at org.junit.runners.ParentRunner.access$000(ParentRunner.java:53)
      at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:229)
      at org.junit.runners.ParentRunner.run(ParentRunner.java:309)
      at org.junit.runners.Suite.runChild(Suite.java:127)
      at org.junit.runners.Suite.runChild(Suite.java:26)
      at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238)
      at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:63)
      at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:236)
      at org.junit.runners.ParentRunner.access$000(ParentRunner.java:53)
      at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:229)
      at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
      at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
      at org.junit.runners.ParentRunner.run(ParentRunner.java:309)
      at org.apache.maven.surefire.junit4.JUnit4Provider.execute(JUnit4Provider.java:264)
      at org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:153)
      at org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:124)
      at org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:200)
      at org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:153)
      at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:103)
      ...Unexpected!

      Attachments

        Activity

          People

            buyingyi Yingyi Bu
            buyingyi Yingyi Bu
            Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: