Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-2964

MutableHashTable fails when spilling partitions without overflow segments

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Critical
    • Resolution: Fixed
    • 0.10.0
    • 0.10.0
    • None
    • None

    Description

      When one performs a join operation with many and large records then the join operation fails with the following exception when it tries to spill a HashPartition.

      java.lang.RuntimeException: Bug in Hybrid Hash Join: Request to spill a partition with less than two buffers.
      	at org.apache.flink.runtime.operators.hash.HashPartition.spillPartition(HashPartition.java:302)
      	at org.apache.flink.runtime.operators.hash.MutableHashTable.spillPartition(MutableHashTable.java:1108)
      	at org.apache.flink.runtime.operators.hash.MutableHashTable.nextSegment(MutableHashTable.java:1277)
      	at org.apache.flink.runtime.operators.hash.HashPartition$BuildSideBuffer.nextSegment(HashPartition.java:524)
      	at org.apache.flink.runtime.memory.AbstractPagedOutputView.advance(AbstractPagedOutputView.java:140)
      	at org.apache.flink.runtime.memory.AbstractPagedOutputView.write(AbstractPagedOutputView.java:201)
      	at org.apache.flink.runtime.memory.AbstractPagedOutputView.write(AbstractPagedOutputView.java:178)
      	at org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer.serialize(BytePrimitiveArraySerializer.java:74)
      	at org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer.serialize(BytePrimitiveArraySerializer.java:30)
      	at org.apache.flink.runtime.operators.hash.HashPartition.insertIntoBuildBuffer(HashPartition.java:257)
      	at org.apache.flink.runtime.operators.hash.MutableHashTable.insertIntoTable(MutableHashTable.java:856)
      	at org.apache.flink.runtime.operators.hash.MutableHashTable.buildInitialTable(MutableHashTable.java:685)
      	at org.apache.flink.runtime.operators.hash.MutableHashTable.open(MutableHashTable.java:443)
      	at org.apache.flink.runtime.operators.hash.HashTableTest.testSpillingWhenBuildingTableWithoutOverflow(HashTableTest.java:234)
      	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
      	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      	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.runner.JUnitCore.run(JUnitCore.java:160)
      	at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:78)
      	at com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:212)
      	at com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:68)
      	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
      	at com.intellij.rt.execution.application.AppMain.main(AppMain.java:140)
      

      The reason is that the HashPartition does not include the number of used memory segments by the BuildSideBuffer when it counts the currently occupied memory segments.

      Attachments

        Issue Links

          Activity

            People

              trohrmann Till Rohrmann
              trohrmann Till Rohrmann
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: