flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-2964) MutableHashTable fails when spilling partitions without overflow segments
Date Wed, 04 Nov 2015 01:21:27 GMT

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

ASF GitHub Bot commented on FLINK-2964:
---------------------------------------

GitHub user tillrohrmann opened a pull request:

    https://github.com/apache/flink/pull/1324

    [FLINK-2964] [runtime] Fix broken spilling of MutableHashTable

    The `HashPartition` did not count properly the number of occupied memory segments, because
it excluded the memory segments of the `BuildSideBuffer`. That caused the `MutableHashTable`
to fail when trying to spill a partition which did not have any overflow segments. This PR
fixes the problem by also counting the memory segments of the `BuildSideBuffer`.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/tillrohrmann/flink fixMutableHashTable

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/1324.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1324
    
----
commit 9db81acd4012cae2a2fc845bc762ab7082463ba2
Author: Till Rohrmann <trohrmann@apache.org>
Date:   2015-11-04T01:17:12Z

    [FLINK-2964] [runtime] Fix broken spilling of MutableHashTable
    
    The HashPartition did not count properly the number of occupied memory segments, because
it excluded the memory segments of the BuildSideBuffer. That caused the MutableHashTable to
fail when trying to spill a partition which did not have any overflow segments. This PR fixes
the problem by also counting the memory segments of the BuildSideBuffer.

----


> MutableHashTable fails when spilling partitions without overflow segments
> -------------------------------------------------------------------------
>
>                 Key: FLINK-2964
>                 URL: https://issues.apache.org/jira/browse/FLINK-2964
>             Project: Flink
>          Issue Type: Bug
>    Affects Versions: 0.10
>            Reporter: Till Rohrmann
>            Assignee: Till Rohrmann
>            Priority: Critical
>
> 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}}.
> {code}
> 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)
> {code}
> 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.



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

Mime
View raw message