Repository: hive
Updated Branches:
  refs/heads/branch-1 476a8e8c9 -> db72a865c


HIVE-18836 : backport HIVE-17600 to hive 1 (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/db72a865
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/db72a865
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/db72a865

Branch: refs/heads/branch-1
Commit: db72a865c8515aa3153e9973c4dd4d3c89d7967a
Parents: 476a8e8
Author: sergey <ser...@apache.org>
Authored: Fri Mar 2 12:45:51 2018 -0800
Committer: sergey <ser...@apache.org>
Committed: Fri Mar 2 12:46:08 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hive/conf/HiveConf.java  |  4 ++++
 .../org/apache/hadoop/hive/ql/io/orc/OrcFile.java   | 10 +++++++++-
 .../org/apache/hadoop/hive/ql/io/orc/OutStream.java | 15 +++++++++++++++
 .../apache/hadoop/hive/ql/io/orc/WriterImpl.java    |  1 +
 .../apache/hadoop/hive/ql/io/orc/TestOutStream.java | 16 ++++++++++++++++
 5 files changed, 45 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/db72a865/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java 
b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index e0c8e82..9b44e30 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2174,6 +2174,10 @@ public class HiveConf extends Configuration {
     HIVEOPTLISTBUCKETING("hive.optimize.listbucketing", false,
         "Enable list bucketing optimizer. Default value is false so that we 
disable it by default."),
 
+    ORC_ENFORCE_COMPRESSION_BUFFER_SIZE("hive.exec.orc.buffer.size.enforce", 
false,
+      "Defines whether to enforce ORC compression buffer size."),
+
+
     // Allow TCP Keep alive socket option for for HiveServer or a maximum 
timeout for the socket.
     SERVER_READ_SOCKET_TIMEOUT("hive.server.read.socket.timeout", "10s",
         new TimeValidator(TimeUnit.SECONDS),

http://git-wip-us.apache.org/repos/asf/hive/blob/db72a865/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java 
b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
index dc00e38..c83b8c1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.io.filters.BloomFilterIO;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 
@@ -161,7 +162,8 @@ public final class OrcFile {
     BLOCK_PADDING("orc.block.padding"),
     ENCODING_STRATEGY("orc.encoding.strategy"),
     BLOOM_FILTER_COLUMNS("orc.bloom.filter.columns"),
-    BLOOM_FILTER_FPP("orc.bloom.filter.fpp");
+    BLOOM_FILTER_FPP("orc.bloom.filter.fpp"),
+    ENFORCE_BUFFER_SIZE("orc.buffer.size.enforce");
 
     private final String propName;
 
@@ -314,6 +316,12 @@ public final class OrcFile {
           : CompressionKind.valueOf(propValue.toUpperCase());
 
       propValue = tableProperties == null ? null
+          : 
tableProperties.getProperty(OrcTableProperties.ENFORCE_BUFFER_SIZE.propName);
+      enforceBufferSize = propValue == null ? HiveConf.getBoolVar(conf, 
ConfVars.ORC_ENFORCE_COMPRESSION_BUFFER_SIZE)
+          : Boolean.parseBoolean(propValue);
+
+
+      propValue = tableProperties == null ? null
           : 
tableProperties.getProperty(OrcTableProperties.BLOOM_FILTER_COLUMNS.propName);
       bloomFilterColumns = propValue;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/db72a865/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OutStream.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OutStream.java 
b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OutStream.java
index e2096eb..32992e9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OutStream.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OutStream.java
@@ -282,5 +282,20 @@ class OutStream extends PositionedOutputStream {
   public boolean isSuppressed() {
     return suppress;
   }
+
+  /**
+   * Throws exception if the bufferSize argument equals or exceeds 2^(3*8 - 1).
+   * See {@link OutStream#writeHeader(ByteBuffer, int, int, boolean)}.
+   * The bufferSize needs to be expressible in 3 bytes, and uses the least 
significant byte
+   * to indicate original/compressed bytes.
+   * @param bufferSize The ORC compression buffer size being checked.
+   * @throws IllegalArgumentException If bufferSize value exceeds threshold.
+   */
+  static void assertBufferSizeValid(int bufferSize) throws 
IllegalArgumentException {
+    if (bufferSize >= (1 << 23)) {
+      throw new IllegalArgumentException("Illegal value of ORC compression 
buffer size: " + bufferSize);
+    }
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/db72a865/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java 
b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
index 3929d7b..fc7c397 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
@@ -210,6 +210,7 @@ public class WriterImpl implements Writer, 
MemoryManager.Callback {
       allColumns = getColumnNamesFromInspector(inspector);
     }
     if (enforceBufferSize) {
+      OutStream.assertBufferSizeValid(bufferSize);
       this.bufferSize = bufferSize;
     } else {
       this.bufferSize = getEstimatedBufferSize(defaultStripeSize,

http://git-wip-us.apache.org/repos/asf/hive/blob/db72a865/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOutStream.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOutStream.java 
b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOutStream.java
index 58c1bfb..a23e246 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOutStream.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOutStream.java
@@ -24,6 +24,7 @@ import org.mockito.Mockito;
 import java.nio.ByteBuffer;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 public class TestOutStream {
     @Test
@@ -38,4 +39,19 @@ public class TestOutStream {
         Mockito.verify(receiver).output(Mockito.any(ByteBuffer.class));
         assertEquals(0L, stream.getBufferSize());
     }
+
+ @Test
+ public void testAssertBufferSizeValid() throws Exception {
+   try {
+     OutStream.assertBufferSizeValid(1 + (1<<23));
+     fail("Invalid buffer-size " + (1 + (1<<23)) + " should have been 
blocked.");
+   }
+   catch (IllegalArgumentException expected) {
+     // Pass.
+   }
+
+   OutStream.assertBufferSizeValid((1<<23) -  1);
+ }
+
+
 }
\ No newline at end of file

Reply via email to