You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2021/02/05 15:28:03 UTC

[GitHub] [hive] szlta opened a new pull request #1951: HIVE-24736

szlta opened a new pull request #1951:
URL: https://github.com/apache/hive/pull/1951


   HIVE-22492 has introduced threadlocal buffers in which LlapCachableBuffer instances are stored before entering LRFU's heap - so that lock contention is eased up.
   
   This is a nice performance improvement, but comes at the cost of losing the exact accounting of llap buffer instances - e.g. if user gives a purge command, not all the cache space is free'd up as one'd expect because purge only considers buffers that the policy knows about. In this case we'd see in LLAP's iomem servlet that the LRFU policy is empty, but a table may still have the full content loaded.
   
   Also, if we use text based tables, during cache load, a set of -OrcEncode threads are used that are ephemeral in nature. Attaching buffers to these threads' thread local structures are ultimately lost. In an edge case we could load lots of data into the cache by reading in many distinct smaller text tables, whose buffers never reach LRFU policy, and hence cache hit ratio will be suffering as a consequence (memory manager will give up asking LRFU to evict, and will free up random buffers).
   
   I propose we try and track the amount of data stored in the BP wrapper threadlocals, and flush them into the heap as a first step of a purge request. This will enhance supportability.
   We should also replace the ephemeral OrcEncode threads with a thread pool, that could actually serve as small performance improvement on its own by saving time and memory to deal with thread lifecycle management.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] szlta commented on a change in pull request #1951: HIVE-24736

Posted by GitBox <gi...@apache.org>.
szlta commented on a change in pull request #1951:
URL: https://github.com/apache/hive/pull/1951#discussion_r571975616



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
##########
@@ -229,11 +230,20 @@ public void debugDumpShort(StringBuilder sb) {
         new LinkedBlockingQueue<Runnable>(),
         new ThreadFactoryBuilder().setNameFormat("IO-Elevator-Thread-%d").setDaemon(true).build());
     FixedSizedObjectPool<IoTrace> tracePool = IoTrace.createTracePool(conf);
+    if (isEncodeEnabled) {
+      int encodeThreads = numThreads * 2;

Review comment:
       Yeah quite arbitrary I'll give you that.
   Text reading is actually started by the "regular" IO threads, and if encoding is needed for cache insertion, than one of these threads can produce more async encode tasks. This may end up being bursty, and according to what I've seen a bigger thread pool could come handy.
   Anyway I made this configurable now.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #1951: HIVE-24736

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1951:
URL: https://github.com/apache/hive/pull/1951#discussion_r571915485



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
##########
@@ -836,13 +912,33 @@ public String description() {
      * @return long array with LRFU stats
      */
     public long[] getUsageStats() {
-      long dataOnHeap = 0L;   // all non-meta related buffers on min-heap
-      long dataOnList = 0L;   // all non-meta related buffers on eviction list
-      long metaOnHeap = 0L;   // meta data buffers on min-heap
-      long metaOnList = 0L;   // meta data buffers on eviction list
-      long listSize   = 0L;   // number of entries on eviction list
-      long lockedData = 0L;   // number of bytes in locked data buffers
-      long lockedMeta = 0L;   // number of bytes in locked metadata buffers
+      long dataOnHeap     = 0L;   // all non-meta related buffers on min-heap
+      long dataOnList     = 0L;   // all non-meta related buffers on eviction list
+      long metaOnHeap     = 0L;   // meta data buffers on min-heap
+      long metaOnList     = 0L;   // meta data buffers on eviction list
+      long listSize       = 0L;   // number of entries on eviction list
+      long lockedData     = 0L;   // number of bytes in locked data buffers
+      long lockedMeta     = 0L;   // number of bytes in locked metadata buffers
+      long bpWrapCount    = 0L;   // number of buffers in BP wrapper threadlocals
+      long bpWrapDistinct = 0L;   // number of distinct buffers in BP wrapper threadlocals
+      long bpWrapData     = 0L;   // number of bytes stored in BP wrapper data buffers
+      long bpWrapMeta     = 0L;   // number of bytes stored in BP wrapper metadata buffers
+
+      // Using set to produce result of distinct buffers only
+      // (same buffer may be present in multiple thread local bp wrappers, or even inside heap/list, but ultimately
+      // it uses the same cache space)
+      Set<LlapCacheableBuffer> bpWrapperBuffers = new HashSet<>();
+      for (BPWrapper bpWrapper : bpWrappers.values()) {
+        bpWrapper.lock.lock();

Review comment:
       How often we call `getUsageStats()` is called? Is it ok to lock all of the buffers here?

##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
##########
@@ -909,19 +1017,24 @@ public synchronized void getMetrics(MetricsCollector collector, boolean all) {
                                           .tag(MsInfo.SessionId, session);
 
       // add the values to the new record
-      mrb.addCounter(PolicyInformation.DataOnHeap,   usageStats[DATAONHEAP])
-          .addCounter(PolicyInformation.DataOnList,  usageStats[DATAONLIST])
-          .addCounter(PolicyInformation.MetaOnHeap,  usageStats[METAONHEAP])
-          .addCounter(PolicyInformation.MetaOnList,  usageStats[METAONLIST])
-          .addCounter(PolicyInformation.DataLocked,  usageStats[LOCKEDDATA])
-          .addCounter(PolicyInformation.MetaLocked,  usageStats[LOCKEDMETA])
-          .addCounter(PolicyInformation.HeapSize,    heapSize)
-          .addCounter(PolicyInformation.HeapSizeMax, maxHeapSize)
-          .addCounter(PolicyInformation.ListSize,    usageStats[LISTSIZE])
-          .addCounter(PolicyInformation.TotalData,   usageStats[DATAONHEAP]
-                                                     + usageStats[DATAONLIST])
-          .addCounter(PolicyInformation.TotalMeta,   usageStats[METAONHEAP]
-                                                     + usageStats[METAONLIST]);
+      mrb.addCounter(PolicyInformation.DataOnHeap,         usageStats[DATAONHEAP])
+          .addCounter(PolicyInformation.DataOnList,        usageStats[DATAONLIST])
+          .addCounter(PolicyInformation.MetaOnHeap,        usageStats[METAONHEAP])
+          .addCounter(PolicyInformation.MetaOnList,        usageStats[METAONLIST])
+          .addCounter(PolicyInformation.DataLocked,        usageStats[LOCKEDDATA])
+          .addCounter(PolicyInformation.MetaLocked,        usageStats[LOCKEDMETA])
+          .addCounter(PolicyInformation.HeapSize,          heapSize)
+          .addCounter(PolicyInformation.HeapSizeMax,       maxHeapSize)
+          .addCounter(PolicyInformation.ListSize,          usageStats[LISTSIZE])
+          .addCounter(PolicyInformation.BPWrapperCount,    usageStats[BPWRAPCNT])
+          .addCounter(PolicyInformation.BPWrapperDistinct, usageStats[BPWRAPDISTINCT])
+          .addCounter(PolicyInformation.BPWrapperData,     usageStats[BPWRAPDATA])
+          .addCounter(PolicyInformation.TotalData,     usageStats[DATAONHEAP]

Review comment:
       nit: Maybe formatting?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] szlta commented on a change in pull request #1951: HIVE-24736

Posted by GitBox <gi...@apache.org>.
szlta commented on a change in pull request #1951:
URL: https://github.com/apache/hive/pull/1951#discussion_r571973578



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
##########
@@ -836,13 +912,33 @@ public String description() {
      * @return long array with LRFU stats
      */
     public long[] getUsageStats() {
-      long dataOnHeap = 0L;   // all non-meta related buffers on min-heap
-      long dataOnList = 0L;   // all non-meta related buffers on eviction list
-      long metaOnHeap = 0L;   // meta data buffers on min-heap
-      long metaOnList = 0L;   // meta data buffers on eviction list
-      long listSize   = 0L;   // number of entries on eviction list
-      long lockedData = 0L;   // number of bytes in locked data buffers
-      long lockedMeta = 0L;   // number of bytes in locked metadata buffers
+      long dataOnHeap     = 0L;   // all non-meta related buffers on min-heap
+      long dataOnList     = 0L;   // all non-meta related buffers on eviction list
+      long metaOnHeap     = 0L;   // meta data buffers on min-heap
+      long metaOnList     = 0L;   // meta data buffers on eviction list
+      long listSize       = 0L;   // number of entries on eviction list
+      long lockedData     = 0L;   // number of bytes in locked data buffers
+      long lockedMeta     = 0L;   // number of bytes in locked metadata buffers
+      long bpWrapCount    = 0L;   // number of buffers in BP wrapper threadlocals
+      long bpWrapDistinct = 0L;   // number of distinct buffers in BP wrapper threadlocals
+      long bpWrapData     = 0L;   // number of bytes stored in BP wrapper data buffers
+      long bpWrapMeta     = 0L;   // number of bytes stored in BP wrapper metadata buffers
+
+      // Using set to produce result of distinct buffers only
+      // (same buffer may be present in multiple thread local bp wrappers, or even inside heap/list, but ultimately
+      // it uses the same cache space)
+      Set<LlapCacheableBuffer> bpWrapperBuffers = new HashSet<>();
+      for (BPWrapper bpWrapper : bpWrappers.values()) {
+        bpWrapper.lock.lock();

Review comment:
       It is only called when someone (e.g. cluster admin) asks for memory stats through web UI. Hence it is okay to block all structures for these cases.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on pull request #1951: HIVE-24736

Posted by GitBox <gi...@apache.org>.
pvary commented on pull request #1951:
URL: https://github.com/apache/hive/pull/1951#issuecomment-775027304


   LGTM. Some questions only on sizing, and locking


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #1951: HIVE-24736

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1951:
URL: https://github.com/apache/hive/pull/1951#discussion_r571917546



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
##########
@@ -229,11 +230,20 @@ public void debugDumpShort(StringBuilder sb) {
         new LinkedBlockingQueue<Runnable>(),
         new ThreadFactoryBuilder().setNameFormat("IO-Elevator-Thread-%d").setDaemon(true).build());
     FixedSizedObjectPool<IoTrace> tracePool = IoTrace.createTracePool(conf);
+    if (isEncodeEnabled) {

Review comment:
       Why did we combine the 2 changes to a single jira?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] szlta commented on a change in pull request #1951: HIVE-24736

Posted by GitBox <gi...@apache.org>.
szlta commented on a change in pull request #1951:
URL: https://github.com/apache/hive/pull/1951#discussion_r571973759



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
##########
@@ -909,19 +1017,24 @@ public synchronized void getMetrics(MetricsCollector collector, boolean all) {
                                           .tag(MsInfo.SessionId, session);
 
       // add the values to the new record
-      mrb.addCounter(PolicyInformation.DataOnHeap,   usageStats[DATAONHEAP])
-          .addCounter(PolicyInformation.DataOnList,  usageStats[DATAONLIST])
-          .addCounter(PolicyInformation.MetaOnHeap,  usageStats[METAONHEAP])
-          .addCounter(PolicyInformation.MetaOnList,  usageStats[METAONLIST])
-          .addCounter(PolicyInformation.DataLocked,  usageStats[LOCKEDDATA])
-          .addCounter(PolicyInformation.MetaLocked,  usageStats[LOCKEDMETA])
-          .addCounter(PolicyInformation.HeapSize,    heapSize)
-          .addCounter(PolicyInformation.HeapSizeMax, maxHeapSize)
-          .addCounter(PolicyInformation.ListSize,    usageStats[LISTSIZE])
-          .addCounter(PolicyInformation.TotalData,   usageStats[DATAONHEAP]
-                                                     + usageStats[DATAONLIST])
-          .addCounter(PolicyInformation.TotalMeta,   usageStats[METAONHEAP]
-                                                     + usageStats[METAONLIST]);
+      mrb.addCounter(PolicyInformation.DataOnHeap,         usageStats[DATAONHEAP])
+          .addCounter(PolicyInformation.DataOnList,        usageStats[DATAONLIST])
+          .addCounter(PolicyInformation.MetaOnHeap,        usageStats[METAONHEAP])
+          .addCounter(PolicyInformation.MetaOnList,        usageStats[METAONLIST])
+          .addCounter(PolicyInformation.DataLocked,        usageStats[LOCKEDDATA])
+          .addCounter(PolicyInformation.MetaLocked,        usageStats[LOCKEDMETA])
+          .addCounter(PolicyInformation.HeapSize,          heapSize)
+          .addCounter(PolicyInformation.HeapSizeMax,       maxHeapSize)
+          .addCounter(PolicyInformation.ListSize,          usageStats[LISTSIZE])
+          .addCounter(PolicyInformation.BPWrapperCount,    usageStats[BPWRAPCNT])
+          .addCounter(PolicyInformation.BPWrapperDistinct, usageStats[BPWRAPDISTINCT])
+          .addCounter(PolicyInformation.BPWrapperData,     usageStats[BPWRAPDATA])
+          .addCounter(PolicyInformation.TotalData,     usageStats[DATAONHEAP]

Review comment:
       Fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] szlta commented on a change in pull request #1951: HIVE-24736

Posted by GitBox <gi...@apache.org>.
szlta commented on a change in pull request #1951:
URL: https://github.com/apache/hive/pull/1951#discussion_r571977069



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
##########
@@ -229,11 +230,20 @@ public void debugDumpShort(StringBuilder sb) {
         new LinkedBlockingQueue<Runnable>(),
         new ThreadFactoryBuilder().setNameFormat("IO-Elevator-Thread-%d").setDaemon(true).build());
     FixedSizedObjectPool<IoTrace> tracePool = IoTrace.createTracePool(conf);
+    if (isEncodeEnabled) {

Review comment:
       Making the encode threads pooled has little advantage on its own, but is required for the accurate buffer accounting part - moreover the two change parts are quite small, so I thought it makes sense to combine them




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #1951: HIVE-24736

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1951:
URL: https://github.com/apache/hive/pull/1951#discussion_r571917349



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
##########
@@ -229,11 +230,20 @@ public void debugDumpShort(StringBuilder sb) {
         new LinkedBlockingQueue<Runnable>(),
         new ThreadFactoryBuilder().setNameFormat("IO-Elevator-Thread-%d").setDaemon(true).build());
     FixedSizedObjectPool<IoTrace> tracePool = IoTrace.createTracePool(conf);
+    if (isEncodeEnabled) {
+      int encodeThreads = numThreads * 2;

Review comment:
       Why exactly 2 times?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #1951: HIVE-24736

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1951:
URL: https://github.com/apache/hive/pull/1951#discussion_r571912244



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
##########
@@ -157,35 +206,29 @@ public void notifyUnlock(LlapCacheableBuffer buffer) {
     if (proactiveEvictionEnabled && !instantProactiveEviction) {
       buffer.removeProactiveEvictionMark();
     }
-    int count = threadLocalCount.get();
-    final LlapCacheableBuffer[] cacheableBuffers = threadLocalBuffers.get() ;
-    if (count < maxQueueSize) {
-      cacheableBuffers[count] = buffer;
-      threadLocalCount.set(++count);
-    }
-    if (count <= maxQueueSize / 2) {
-      // case too early to flush
-      return;
-    }
+    BPWrapper bpWrapper = threadLocalBPWrapper.get();
 
-    if (count == maxQueueSize) {
-      // case we have to flush thus block on heap lock
-      heapLock.lock();
-      try {
-        doNotifyUnderHeapLock(count, cacheableBuffers);
-      } finally {
-        threadLocalCount.set(0);
-        heapLock.unlock();
+    // This will only block in a very very rare scenario only.
+    bpWrapper.lock.lock();
+    try {
+      final LlapCacheableBuffer[] cacheableBuffers = bpWrapper.buffers;
+      if (bpWrapper.count < maxQueueSize) {
+        cacheableBuffers[bpWrapper.count] = buffer;
+        ++bpWrapper.count;
       }
-      return;
-    }
-    if (heapLock.tryLock()) {
-      try {
-        doNotifyUnderHeapLock(count, cacheableBuffers);
-      } finally {
-        threadLocalCount.set(0);
-        heapLock.unlock();
+      if (bpWrapper.count <= maxQueueSize / 2) {
+        // case too early to flush
+        return;
+      }
+
+      if (bpWrapper.count == maxQueueSize) {
+        // case we have to flush thus block on heap lock
+        bpWrapper.flush();
+        return;
       }
+      bpWrapper.tryFlush(); //case 50% < queue usage < 100%, flush is preferred but not required yet

Review comment:
       QQ: How did you come up with these numbers? Any tests showing these are the good values?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] pvary commented on a change in pull request #1951: HIVE-24736

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #1951:
URL: https://github.com/apache/hive/pull/1951#discussion_r571923233



##########
File path: llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
##########
@@ -157,35 +206,29 @@ public void notifyUnlock(LlapCacheableBuffer buffer) {
     if (proactiveEvictionEnabled && !instantProactiveEviction) {
       buffer.removeProactiveEvictionMark();
     }
-    int count = threadLocalCount.get();
-    final LlapCacheableBuffer[] cacheableBuffers = threadLocalBuffers.get() ;
-    if (count < maxQueueSize) {
-      cacheableBuffers[count] = buffer;
-      threadLocalCount.set(++count);
-    }
-    if (count <= maxQueueSize / 2) {
-      // case too early to flush
-      return;
-    }
+    BPWrapper bpWrapper = threadLocalBPWrapper.get();
 
-    if (count == maxQueueSize) {
-      // case we have to flush thus block on heap lock
-      heapLock.lock();
-      try {
-        doNotifyUnderHeapLock(count, cacheableBuffers);
-      } finally {
-        threadLocalCount.set(0);
-        heapLock.unlock();
+    // This will only block in a very very rare scenario only.
+    bpWrapper.lock.lock();
+    try {
+      final LlapCacheableBuffer[] cacheableBuffers = bpWrapper.buffers;
+      if (bpWrapper.count < maxQueueSize) {
+        cacheableBuffers[bpWrapper.count] = buffer;
+        ++bpWrapper.count;
       }
-      return;
-    }
-    if (heapLock.tryLock()) {
-      try {
-        doNotifyUnderHeapLock(count, cacheableBuffers);
-      } finally {
-        threadLocalCount.set(0);
-        heapLock.unlock();
+      if (bpWrapper.count <= maxQueueSize / 2) {
+        // case too early to flush
+        return;
+      }
+
+      if (bpWrapper.count == maxQueueSize) {
+        // case we have to flush thus block on heap lock
+        bpWrapper.flush();
+        return;
       }
+      bpWrapper.tryFlush(); //case 50% < queue usage < 100%, flush is preferred but not required yet

Review comment:
       That is a broken link now 😄
   But fair enough




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] szlta merged pull request #1951: HIVE-24736

Posted by GitBox <gi...@apache.org>.
szlta merged pull request #1951:
URL: https://github.com/apache/hive/pull/1951


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org