You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2018/04/03 06:50:45 UTC

[kylin] 07/10: KYLIN-3324 fix NegativeArraySizeException in CreateDictionaryJob$2.getDictionary()

This is an automated email from the ASF dual-hosted git repository.

liyang pushed a commit to branch sync
in repository https://gitbox.apache.org/repos/asf/kylin.git

commit 08a0b9295088a91ac6b34ed3951ed8fb8404ee04
Author: Li Yang <li...@apache.org>
AuthorDate: Thu Mar 29 06:04:16 2018 +0800

    KYLIN-3324 fix NegativeArraySizeException in CreateDictionaryJob$2.getDictionary()
---
 .../java/org/apache/kylin/engine/mr/steps/CreateDictionaryJob.java  | 6 +++---
 .../org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java    | 3 ++-
 .../apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java    | 3 ++-
 .../java/org/apache/kylin/engine/mr/steps/UHCDictionaryJob.java     | 4 ++--
 .../java/org/apache/kylin/engine/mr/steps/UHCDictionaryReducer.java | 3 ++-
 5 files changed, 11 insertions(+), 8 deletions(-)

diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CreateDictionaryJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CreateDictionaryJob.java
index dab4880..e01da9e 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CreateDictionaryJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CreateDictionaryJob.java
@@ -26,7 +26,7 @@ import java.util.List;
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.ArrayPrimitiveWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.SequenceFile;
 import org.apache.hadoop.util.ToolRunner;
@@ -97,10 +97,10 @@ public class CreateDictionaryJob extends AbstractHadoopJob {
 
                 try (SequenceFile.Reader reader = new SequenceFile.Reader(HadoopUtil.getCurrentConfiguration(), SequenceFile.Reader.file(dictFile))) {
                     NullWritable key = NullWritable.get();
-                    BytesWritable value = new BytesWritable();
+                    ArrayPrimitiveWritable value = new ArrayPrimitiveWritable();
                     reader.next(key, value);
 
-                    ByteBuffer buffer = new ByteArray(value.getBytes()).asBuffer();
+                    ByteBuffer buffer = new ByteArray((byte[]) value.get()).asBuffer();
                     try (DataInputStream is = new DataInputStream(new ByteBufferBackedInputStream(buffer))) {
                         String dictClassName = is.readUTF();
                         Dictionary<String> dict = (Dictionary<String>) ClassUtil.newInstance(dictClassName);
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java
index cc4f260..f96944a 100755
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsJob.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.ArrayPrimitiveWritable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;
@@ -144,7 +145,7 @@ public class FactDistinctColumnsJob extends AbstractHadoopJob {
 
         // make each reducer output to respective dir
         MultipleOutputs.addNamedOutput(job, BatchConstants.CFG_OUTPUT_COLUMN, SequenceFileOutputFormat.class, NullWritable.class, Text.class);
-        MultipleOutputs.addNamedOutput(job, BatchConstants.CFG_OUTPUT_DICT, SequenceFileOutputFormat.class, NullWritable.class, BytesWritable.class);
+        MultipleOutputs.addNamedOutput(job, BatchConstants.CFG_OUTPUT_DICT, SequenceFileOutputFormat.class, NullWritable.class, ArrayPrimitiveWritable.class);
         MultipleOutputs.addNamedOutput(job, BatchConstants.CFG_OUTPUT_STATISTICS, SequenceFileOutputFormat.class, LongWritable.class, BytesWritable.class);
         MultipleOutputs.addNamedOutput(job, BatchConstants.CFG_OUTPUT_PARTITION, TextOutputFormat.class, NullWritable.class, LongWritable.class);
 
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java
index cad947c..801771a 100755
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsReducer.java
@@ -27,6 +27,7 @@ import java.util.Map;
 
 import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.ArrayPrimitiveWritable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.NullWritable;
@@ -239,7 +240,7 @@ public class FactDistinctColumnsReducer extends KylinReducer<SelfDefineSortableK
             outputStream.writeUTF(dict.getClass().getName());
             dict.write(outputStream);
 
-            mos.write(BatchConstants.CFG_OUTPUT_DICT, NullWritable.get(), new BytesWritable(baos.toByteArray()), dictFileName);
+            mos.write(BatchConstants.CFG_OUTPUT_DICT, NullWritable.get(), new ArrayPrimitiveWritable(baos.toByteArray()), dictFileName);
         }
     }
 
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UHCDictionaryJob.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UHCDictionaryJob.java
index 1b1a7f0..0903228 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UHCDictionaryJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UHCDictionaryJob.java
@@ -24,7 +24,7 @@ import java.util.Map;
 
 import org.apache.commons.cli.Options;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.ArrayPrimitiveWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
@@ -131,7 +131,7 @@ public class UHCDictionaryJob extends AbstractHadoopJob {
         job.setPartitionerClass(UHCDictionaryPartitioner.class);
         job.setNumReduceTasks(numberOfReducers);
 
-        MultipleOutputs.addNamedOutput(job, BatchConstants.CFG_OUTPUT_DICT, SequenceFileOutputFormat.class, NullWritable.class, BytesWritable.class);
+        MultipleOutputs.addNamedOutput(job, BatchConstants.CFG_OUTPUT_DICT, SequenceFileOutputFormat.class, NullWritable.class, ArrayPrimitiveWritable.class);
         FileOutputFormat.setOutputPath(job, output);
         job.getConfiguration().set(BatchConstants.CFG_OUTPUT_PATH, output.toString());
 
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UHCDictionaryReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UHCDictionaryReducer.java
index 6da198d..d9a3549 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UHCDictionaryReducer.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/UHCDictionaryReducer.java
@@ -26,6 +26,7 @@ import java.util.List;
 
 import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.ArrayPrimitiveWritable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.Text;
@@ -106,7 +107,7 @@ public class UHCDictionaryReducer extends KylinReducer<SelfDefineSortableKey, Nu
             outputStream.writeUTF(dict.getClass().getName());
             dict.write(outputStream);
 
-            mos.write(BatchConstants.CFG_OUTPUT_DICT, NullWritable.get(), new BytesWritable(baos.toByteArray()), dictFileName);
+            mos.write(BatchConstants.CFG_OUTPUT_DICT, NullWritable.get(), new ArrayPrimitiveWritable(baos.toByteArray()), dictFileName);
         }
         mos.close();
     }

-- 
To stop receiving notification emails like this one, please contact
liyang@apache.org.