You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2017/04/27 20:07:49 UTC

[1/8] hive git commit: HIVE-16547 : LLAP: may not unlock buffers in some cases (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Repository: hive
Updated Branches:
  refs/heads/branch-2 96784f7ca -> 4025eddd4
  refs/heads/branch-2.2 9bedccc4b -> 3403535d3
  refs/heads/branch-2.3 346f7086c -> 79339c158
  refs/heads/master 47c714167 -> 81be9d0d2


HIVE-16547 : LLAP: may not unlock buffers in some cases (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f4d017b8
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f4d017b8
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f4d017b8

Branch: refs/heads/master
Commit: f4d017b801ea256bf076160a31dba88e61c80422
Parents: 47c7141
Author: sergey <se...@apache.org>
Authored: Thu Apr 27 11:52:07 2017 -0700
Committer: sergey <se...@apache.org>
Committed: Thu Apr 27 11:52:07 2017 -0700

----------------------------------------------------------------------
 .../ql/io/orc/encoded/EncodedReaderImpl.java    | 24 ++++++++++++++++----
 1 file changed, 19 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f4d017b8/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
index 326b4b6..5b2e9b5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
@@ -416,8 +416,8 @@ class EncodedReaderImpl implements EncodedReader {
               if (sctx.stripeLevelStream == null) {
                 sctx.stripeLevelStream = POOLS.csdPool.take();
                 // We will be using this for each RG while also sending RGs to processing.
-                // To avoid buffers being unlocked, run refcount one ahead; we will not increase
-                // it when building the last RG, so each RG processing will decref once, and the
+                // To avoid buffers being unlocked, run refcount one ahead; so each RG 
+                 // processing will decref once, and the
                 // last one will unlock the buffers.
                 sctx.stripeLevelStream.incRef();
                 // For stripe-level streams we don't need the extra refcount on the block.
@@ -430,9 +430,7 @@ class EncodedReaderImpl implements EncodedReader {
                   iter = lastCached;
                 }
               }
-              if (!isLastRg) {
-                sctx.stripeLevelStream.incRef();
-              }
+              sctx.stripeLevelStream.incRef();
               cb = sctx.stripeLevelStream;
             } else {
               // This stream can be separated by RG using index. Let's do that.
@@ -479,6 +477,22 @@ class EncodedReaderImpl implements EncodedReader {
     }
 
     // Release the unreleased buffers. See class comment about refcounts.
+    for (int colIx = 0; colIx < colCtxs.length; ++colIx) {
+      ColumnReadContext ctx = colCtxs[colIx];
+      if (ctx == null) continue; // This column is not included.
+      for (int streamIx = 0; streamIx < ctx.streamCount; ++streamIx) {
+        StreamContext sctx = ctx.streams[streamIx];
+        if (sctx == null || sctx.stripeLevelStream == null) continue;
+        if (0 != sctx.stripeLevelStream.decRef()) continue;
+        for (MemoryBuffer buf : sctx.stripeLevelStream.getCacheBuffers()) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Unlocking {} at the end of processing", buf);
+          }
+          cacheWrapper.releaseBuffer(buf);
+        }
+      }
+    }
+
     releaseInitialRefcounts(toRead.next);
     // Release buffers as we are done with all the streams... also see toRelease comment.
     if (toRelease != null) {


[4/8] hive git commit: HIVE-16547 : LLAP: may not unlock buffers in some cases (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-16547 : LLAP: may not unlock buffers in some cases (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/2decd512
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2decd512
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2decd512

Branch: refs/heads/branch-2
Commit: 2decd51234f3d6802f43dab5570520a5d0df036f
Parents: 96784f7
Author: sergey <se...@apache.org>
Authored: Thu Apr 27 11:52:07 2017 -0700
Committer: sergey <se...@apache.org>
Committed: Thu Apr 27 12:44:01 2017 -0700

----------------------------------------------------------------------
 .../ql/io/orc/encoded/EncodedReaderImpl.java    | 24 ++++++++++++++++----
 1 file changed, 19 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/2decd512/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
index 326b4b6..5b2e9b5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
@@ -416,8 +416,8 @@ class EncodedReaderImpl implements EncodedReader {
               if (sctx.stripeLevelStream == null) {
                 sctx.stripeLevelStream = POOLS.csdPool.take();
                 // We will be using this for each RG while also sending RGs to processing.
-                // To avoid buffers being unlocked, run refcount one ahead; we will not increase
-                // it when building the last RG, so each RG processing will decref once, and the
+                // To avoid buffers being unlocked, run refcount one ahead; so each RG 
+                 // processing will decref once, and the
                 // last one will unlock the buffers.
                 sctx.stripeLevelStream.incRef();
                 // For stripe-level streams we don't need the extra refcount on the block.
@@ -430,9 +430,7 @@ class EncodedReaderImpl implements EncodedReader {
                   iter = lastCached;
                 }
               }
-              if (!isLastRg) {
-                sctx.stripeLevelStream.incRef();
-              }
+              sctx.stripeLevelStream.incRef();
               cb = sctx.stripeLevelStream;
             } else {
               // This stream can be separated by RG using index. Let's do that.
@@ -479,6 +477,22 @@ class EncodedReaderImpl implements EncodedReader {
     }
 
     // Release the unreleased buffers. See class comment about refcounts.
+    for (int colIx = 0; colIx < colCtxs.length; ++colIx) {
+      ColumnReadContext ctx = colCtxs[colIx];
+      if (ctx == null) continue; // This column is not included.
+      for (int streamIx = 0; streamIx < ctx.streamCount; ++streamIx) {
+        StreamContext sctx = ctx.streams[streamIx];
+        if (sctx == null || sctx.stripeLevelStream == null) continue;
+        if (0 != sctx.stripeLevelStream.decRef()) continue;
+        for (MemoryBuffer buf : sctx.stripeLevelStream.getCacheBuffers()) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Unlocking {} at the end of processing", buf);
+          }
+          cacheWrapper.releaseBuffer(buf);
+        }
+      }
+    }
+
     releaseInitialRefcounts(toRead.next);
     // Release buffers as we are done with all the streams... also see toRelease comment.
     if (toRelease != null) {


[5/8] hive git commit: HIVE-16547 : LLAP: may not unlock buffers in some cases (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-16547 : LLAP: may not unlock buffers in some cases (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f71d446d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f71d446d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f71d446d

Branch: refs/heads/branch-2.3
Commit: f71d446d4d3bcf77e3249e2a68cff7366c5a1184
Parents: 346f708
Author: sergey <se...@apache.org>
Authored: Thu Apr 27 11:52:07 2017 -0700
Committer: sergey <se...@apache.org>
Committed: Thu Apr 27 12:44:10 2017 -0700

----------------------------------------------------------------------
 .../ql/io/orc/encoded/EncodedReaderImpl.java    | 24 ++++++++++++++++----
 1 file changed, 19 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/f71d446d/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
index 326b4b6..5b2e9b5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
@@ -416,8 +416,8 @@ class EncodedReaderImpl implements EncodedReader {
               if (sctx.stripeLevelStream == null) {
                 sctx.stripeLevelStream = POOLS.csdPool.take();
                 // We will be using this for each RG while also sending RGs to processing.
-                // To avoid buffers being unlocked, run refcount one ahead; we will not increase
-                // it when building the last RG, so each RG processing will decref once, and the
+                // To avoid buffers being unlocked, run refcount one ahead; so each RG 
+                 // processing will decref once, and the
                 // last one will unlock the buffers.
                 sctx.stripeLevelStream.incRef();
                 // For stripe-level streams we don't need the extra refcount on the block.
@@ -430,9 +430,7 @@ class EncodedReaderImpl implements EncodedReader {
                   iter = lastCached;
                 }
               }
-              if (!isLastRg) {
-                sctx.stripeLevelStream.incRef();
-              }
+              sctx.stripeLevelStream.incRef();
               cb = sctx.stripeLevelStream;
             } else {
               // This stream can be separated by RG using index. Let's do that.
@@ -479,6 +477,22 @@ class EncodedReaderImpl implements EncodedReader {
     }
 
     // Release the unreleased buffers. See class comment about refcounts.
+    for (int colIx = 0; colIx < colCtxs.length; ++colIx) {
+      ColumnReadContext ctx = colCtxs[colIx];
+      if (ctx == null) continue; // This column is not included.
+      for (int streamIx = 0; streamIx < ctx.streamCount; ++streamIx) {
+        StreamContext sctx = ctx.streams[streamIx];
+        if (sctx == null || sctx.stripeLevelStream == null) continue;
+        if (0 != sctx.stripeLevelStream.decRef()) continue;
+        for (MemoryBuffer buf : sctx.stripeLevelStream.getCacheBuffers()) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Unlocking {} at the end of processing", buf);
+          }
+          cacheWrapper.releaseBuffer(buf);
+        }
+      }
+    }
+
     releaseInitialRefcounts(toRead.next);
     // Release buffers as we are done with all the streams... also see toRelease comment.
     if (toRelease != null) {


[6/8] hive git commit: HIVE-16545 : LLAP: bug in arena size determination logic (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-16545 : LLAP: bug in arena size determination logic (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/79339c15
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/79339c15
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/79339c15

Branch: refs/heads/branch-2.3
Commit: 79339c158e0d545ad0c9526d6689abba0a0cd6ee
Parents: f71d446
Author: sergey <se...@apache.org>
Authored: Thu Apr 27 11:53:25 2017 -0700
Committer: sergey <se...@apache.org>
Committed: Thu Apr 27 12:44:10 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java  | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/79339c15/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
index e71a09e..3192dca 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
@@ -123,7 +123,8 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
     } else {
       cacheDir = null;
     }
-    int arenaSizeVal = (arenaCount == 0) ? MAX_ARENA_SIZE : (int)(maxSizeVal / arenaCount);
+    long arenaSizeVal = (arenaCount == 0) ? MAX_ARENA_SIZE : maxSizeVal / arenaCount;
+    // The math.min, and the fact that maxAllocation is an int, ensures we don't overflow.
     arenaSizeVal = Math.max(maxAllocation, Math.min(arenaSizeVal, MAX_ARENA_SIZE));
     if (LlapIoImpl.LOG.isInfoEnabled()) {
       LlapIoImpl.LOG.info("Buddy allocator with " + (isDirect ? "direct" : "byte") + " buffers; "
@@ -152,7 +153,7 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
       LlapIoImpl.LOG.warn("Rounding arena size to " + arenaSizeVal + " from " + oldArenaSize
           + " to be divisible by allocation size " + maxAllocation);
     }
-    arenaSize = arenaSizeVal;
+    arenaSize = (int)arenaSizeVal;
     if ((maxSizeVal % arenaSize) > 0) {
       long oldMaxSize = maxSizeVal;
       maxSizeVal = (maxSizeVal / arenaSize) * arenaSize;


[8/8] hive git commit: HIVE-16545 : LLAP: bug in arena size determination logic (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-16545 : LLAP: bug in arena size determination logic (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3403535d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3403535d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3403535d

Branch: refs/heads/branch-2.2
Commit: 3403535d32ed914c99712d38ef2d04f6c8a6a7d2
Parents: c84468e
Author: sergey <se...@apache.org>
Authored: Thu Apr 27 11:53:25 2017 -0700
Committer: sergey <se...@apache.org>
Committed: Thu Apr 27 12:44:20 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java  | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3403535d/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
index 95cc11e..7ed52b0 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
@@ -118,7 +118,8 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
     } else {
       cacheDir = null;
     }
-    int arenaSizeVal = (arenaCount == 0) ? MAX_ARENA_SIZE : (int)(maxSizeVal / arenaCount);
+    long arenaSizeVal = (arenaCount == 0) ? MAX_ARENA_SIZE : maxSizeVal / arenaCount;
+    // The math.min, and the fact that maxAllocation is an int, ensures we don't overflow.
     arenaSizeVal = Math.max(maxAllocation, Math.min(arenaSizeVal, MAX_ARENA_SIZE));
     if (LlapIoImpl.LOG.isInfoEnabled()) {
       LlapIoImpl.LOG.info("Buddy allocator with " + (isDirect ? "direct" : "byte") + " buffers;"
@@ -147,7 +148,7 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
       LlapIoImpl.LOG.warn("Rounding arena size to " + arenaSizeVal + " from " + oldArenaSize
           + " to be divisible by allocation size " + maxAllocation);
     }
-    arenaSize = arenaSizeVal;
+    arenaSize = (int)arenaSizeVal;
     if ((maxSizeVal % arenaSize) > 0) {
       long oldMaxSize = maxSizeVal;
       maxSizeVal = (maxSizeVal / arenaSize) * arenaSize;


[7/8] hive git commit: HIVE-16547 : LLAP: may not unlock buffers in some cases (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-16547 : LLAP: may not unlock buffers in some cases (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/c84468e3
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/c84468e3
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/c84468e3

Branch: refs/heads/branch-2.2
Commit: c84468e3567261181df8fbbf86ac39fd2242ea71
Parents: 9bedccc
Author: sergey <se...@apache.org>
Authored: Thu Apr 27 11:52:07 2017 -0700
Committer: sergey <se...@apache.org>
Committed: Thu Apr 27 12:44:20 2017 -0700

----------------------------------------------------------------------
 .../ql/io/orc/encoded/EncodedReaderImpl.java    | 24 ++++++++++++++++----
 1 file changed, 19 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/c84468e3/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
index 9b31988..3e26d9a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
@@ -374,8 +374,8 @@ class EncodedReaderImpl implements EncodedReader {
               if (sctx.stripeLevelStream == null) {
                 sctx.stripeLevelStream = POOLS.csdPool.take();
                 // We will be using this for each RG while also sending RGs to processing.
-                // To avoid buffers being unlocked, run refcount one ahead; we will not increase
-                // it when building the last RG, so each RG processing will decref once, and the
+                // To avoid buffers being unlocked, run refcount one ahead; so each RG 
+                 // processing will decref once, and the
                 // last one will unlock the buffers.
                 sctx.stripeLevelStream.incRef();
                 // For stripe-level streams we don't need the extra refcount on the block.
@@ -388,9 +388,7 @@ class EncodedReaderImpl implements EncodedReader {
                   iter = lastCached;
                 }
               }
-              if (!isLastRg) {
-                sctx.stripeLevelStream.incRef();
-              }
+              sctx.stripeLevelStream.incRef();
               cb = sctx.stripeLevelStream;
             } else {
               // This stream can be separated by RG using index. Let's do that.
@@ -437,6 +435,22 @@ class EncodedReaderImpl implements EncodedReader {
     }
 
     // Release the unreleased buffers. See class comment about refcounts.
+    for (int colIx = 0; colIx < colCtxs.length; ++colIx) {
+      ColumnReadContext ctx = colCtxs[colIx];
+      if (ctx == null) continue; // This column is not included.
+      for (int streamIx = 0; streamIx < ctx.streamCount; ++streamIx) {
+        StreamContext sctx = ctx.streams[streamIx];
+        if (sctx == null || sctx.stripeLevelStream == null) continue;
+        if (0 != sctx.stripeLevelStream.decRef()) continue;
+        for (MemoryBuffer buf : sctx.stripeLevelStream.getCacheBuffers()) {
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Unlocking {} at the end of processing", buf);
+          }
+          cacheWrapper.releaseBuffer(buf);
+        }
+      }
+    }
+
     releaseInitialRefcounts(toRead.next);
     releaseCacheChunksIntoObjectPool(toRead.next);
   }


[2/8] hive git commit: HIVE-16545 : LLAP: bug in arena size determination logic (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-16545 : LLAP: bug in arena size determination logic (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/81be9d0d
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/81be9d0d
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/81be9d0d

Branch: refs/heads/master
Commit: 81be9d0d2e84bf0cb2791b7fe1478e8c9ece89c8
Parents: f4d017b
Author: sergey <se...@apache.org>
Authored: Thu Apr 27 11:53:25 2017 -0700
Committer: sergey <se...@apache.org>
Committed: Thu Apr 27 11:53:25 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java  | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/81be9d0d/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
index c049d27..302918a 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
@@ -124,7 +124,8 @@ public final class BuddyAllocator
     } else {
       cacheDir = null;
     }
-    int arenaSizeVal = (arenaCount == 0) ? MAX_ARENA_SIZE : (int)(maxSizeVal / arenaCount);
+    long arenaSizeVal = (arenaCount == 0) ? MAX_ARENA_SIZE : maxSizeVal / arenaCount;
+    // The math.min, and the fact that maxAllocation is an int, ensures we don't overflow.
     arenaSizeVal = Math.max(maxAllocation, Math.min(arenaSizeVal, MAX_ARENA_SIZE));
     if (LlapIoImpl.LOG.isInfoEnabled()) {
       LlapIoImpl.LOG.info("Buddy allocator with " + (isDirect ? "direct" : "byte") + " buffers; "
@@ -153,7 +154,7 @@ public final class BuddyAllocator
       LlapIoImpl.LOG.warn("Rounding arena size to " + arenaSizeVal + " from " + oldArenaSize
           + " to be divisible by allocation size " + maxAllocation);
     }
-    arenaSize = arenaSizeVal;
+    arenaSize = (int)arenaSizeVal;
     if ((maxSizeVal % arenaSize) > 0) {
       long oldMaxSize = maxSizeVal;
       maxSizeVal = (maxSizeVal / arenaSize) * arenaSize;


[3/8] hive git commit: HIVE-16545 : LLAP: bug in arena size determination logic (Sergey Shelukhin, reviewed by Prasanth Jayachandran)

Posted by se...@apache.org.
HIVE-16545 : LLAP: bug in arena size determination logic (Sergey Shelukhin, reviewed by Prasanth Jayachandran)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/4025eddd
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/4025eddd
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/4025eddd

Branch: refs/heads/branch-2
Commit: 4025eddd43af1a1ab01b83c1701455d7eb26f100
Parents: 2decd51
Author: sergey <se...@apache.org>
Authored: Thu Apr 27 11:53:25 2017 -0700
Committer: sergey <se...@apache.org>
Committed: Thu Apr 27 12:44:01 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java  | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4025eddd/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
----------------------------------------------------------------------
diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
index e71a09e..3192dca 100644
--- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
+++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/BuddyAllocator.java
@@ -123,7 +123,8 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
     } else {
       cacheDir = null;
     }
-    int arenaSizeVal = (arenaCount == 0) ? MAX_ARENA_SIZE : (int)(maxSizeVal / arenaCount);
+    long arenaSizeVal = (arenaCount == 0) ? MAX_ARENA_SIZE : maxSizeVal / arenaCount;
+    // The math.min, and the fact that maxAllocation is an int, ensures we don't overflow.
     arenaSizeVal = Math.max(maxAllocation, Math.min(arenaSizeVal, MAX_ARENA_SIZE));
     if (LlapIoImpl.LOG.isInfoEnabled()) {
       LlapIoImpl.LOG.info("Buddy allocator with " + (isDirect ? "direct" : "byte") + " buffers; "
@@ -152,7 +153,7 @@ public final class BuddyAllocator implements EvictionAwareAllocator, BuddyAlloca
       LlapIoImpl.LOG.warn("Rounding arena size to " + arenaSizeVal + " from " + oldArenaSize
           + " to be divisible by allocation size " + maxAllocation);
     }
-    arenaSize = arenaSizeVal;
+    arenaSize = (int)arenaSizeVal;
     if ((maxSizeVal % arenaSize) > 0) {
       long oldMaxSize = maxSizeVal;
       maxSizeVal = (maxSizeVal / arenaSize) * arenaSize;