You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by ni...@apache.org on 2020/02/20 16:21:08 UTC
[kylin] 05/06: minor,
Iterator.next() methods should throw NoSuchElementException
This is an automated email from the ASF dual-hosted git repository.
nic pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kylin.git
commit 5d9f4965c206fd62c91fe2055fc0aace5d740659
Author: etherge <et...@163.com>
AuthorDate: Fri Feb 14 18:57:48 2020 -0500
minor, Iterator.next() methods should throw NoSuchElementException
---
.../RecordConsumeBlockingQueueController.java | 18 +++++++++++-------
.../org/apache/kylin/gridtable/GTAggregateScanner.java | 3 ++-
.../apache/kylin/source/datagen/ColumnGenerator.java | 6 +++---
.../gtrecord/SortedIteratorMergerWithLimit.java | 3 ++-
.../cube/v2/coprocessor/endpoint/CubeVisitService.java | 3 ++-
.../columnar/FSInputGeneralColumnDataReader.java | 3 ++-
.../compress/FSInputLZ4CompressedColumnReader.java | 9 +++++----
7 files changed, 27 insertions(+), 18 deletions(-)
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/RecordConsumeBlockingQueueController.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/RecordConsumeBlockingQueueController.java
index 5fc3e32..799c340 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/RecordConsumeBlockingQueueController.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/RecordConsumeBlockingQueueController.java
@@ -18,17 +18,19 @@
package org.apache.kylin.cube.inmemcubing;
+import java.util.NoSuchElementException;
import java.util.concurrent.BlockingQueue;
public class RecordConsumeBlockingQueueController<T> extends ConsumeBlockingQueueController<T> {
public final InputConverterUnit<T> inputConverterUnit;
- private RecordConsumeBlockingQueueController(InputConverterUnit<T> inputConverterUnit, BlockingQueue<T> input, int batchSize) {
+ private RecordConsumeBlockingQueueController(InputConverterUnit<T> inputConverterUnit, BlockingQueue<T> input,
+ int batchSize) {
super(input, batchSize);
this.inputConverterUnit = inputConverterUnit;
}
-
+
private T currentObject = null;
private volatile boolean ifEnd = false;
@@ -59,7 +61,7 @@ public class RecordConsumeBlockingQueueController<T> extends ConsumeBlockingQueu
@Override
public T next() {
if (ifEnd() || currentObject == null)
- throw new IllegalStateException();
+ throw new NoSuchElementException();
T result = currentObject;
currentObject = null;
@@ -69,12 +71,14 @@ public class RecordConsumeBlockingQueueController<T> extends ConsumeBlockingQueu
public boolean ifEnd() {
return ifEnd;
}
-
- public static <T> RecordConsumeBlockingQueueController<T> getQueueController(InputConverterUnit<T> inputConverterUnit, BlockingQueue<T> input){
+
+ public static <T> RecordConsumeBlockingQueueController<T> getQueueController(
+ InputConverterUnit<T> inputConverterUnit, BlockingQueue<T> input) {
return new RecordConsumeBlockingQueueController<>(inputConverterUnit, input, DEFAULT_BATCH_SIZE);
}
-
- public static <T> RecordConsumeBlockingQueueController<T> getQueueController(InputConverterUnit<T> inputConverterUnit, BlockingQueue<T> input, int batchSize){
+
+ public static <T> RecordConsumeBlockingQueueController<T> getQueueController(
+ InputConverterUnit<T> inputConverterUnit, BlockingQueue<T> input, int batchSize) {
return new RecordConsumeBlockingQueueController<>(inputConverterUnit, input, batchSize);
}
}
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
index 78e413a..06c6646 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
@@ -35,6 +35,7 @@ import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.Map.Entry;
+import java.util.NoSuchElementException;
import java.util.PriorityQueue;
import java.util.SortedMap;
import java.util.TreeMap;
@@ -704,7 +705,7 @@ public class GTAggregateScanner implements IGTScanner, IGTBypassChecker {
dis.readFully(value);
return new Pair<>(key, value);
} catch (Exception e) {
- throw new RuntimeException(
+ throw new NoSuchElementException(
"Cannot read AggregationCache from dumped file: " + e.getMessage());
}
}
diff --git a/core-metadata/src/main/java/org/apache/kylin/source/datagen/ColumnGenerator.java b/core-metadata/src/main/java/org/apache/kylin/source/datagen/ColumnGenerator.java
index 2c4338b..81223be 100644
--- a/core-metadata/src/main/java/org/apache/kylin/source/datagen/ColumnGenerator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/source/datagen/ColumnGenerator.java
@@ -255,8 +255,8 @@ public class ColumnGenerator {
@Override
public String next() {
if (values.isEmpty())
- return null;
-
+ throw new NoSuchElementException();
+
return values.get(rand.nextInt(values.size()));
}
}
@@ -318,7 +318,7 @@ public class ColumnGenerator {
if (input.hasNext()) {
r = input.next();
}
-
+
if (rand.nextDouble() < nullPct) {
r = nullStr;
}
diff --git a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java
index 9baee14..123e4be 100644
--- a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java
+++ b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java
@@ -21,6 +21,7 @@ package org.apache.kylin.storage.gtrecord;
import java.lang.reflect.InvocationTargetException;
import java.util.Comparator;
import java.util.Iterator;
+import java.util.NoSuchElementException;
import java.util.PriorityQueue;
import com.google.common.base.Preconditions;
@@ -123,7 +124,7 @@ public class SortedIteratorMergerWithLimit<E extends Cloneable> extends SortedIt
@Override
public E next() {
if (!nextFetched) {
- throw new IllegalStateException("Should hasNext() before next()");
+ throw new NoSuchElementException("Should hasNext() before next()");
}
//TODO: remove this check when validated
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
index f0a8eb9..c6e5d00 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
@@ -26,6 +26,7 @@ import java.nio.BufferOverflowException;
import java.nio.ByteBuffer;
import java.util.Iterator;
import java.util.List;
+import java.util.NoSuchElementException;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang.ArrayUtils;
@@ -124,7 +125,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
public List<Cell> next() {
if (nextOne.size() < 1) {
- throw new IllegalStateException();
+ throw new NoSuchElementException();
}
ret.clear();
ret.addAll(nextOne);
diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FSInputGeneralColumnDataReader.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FSInputGeneralColumnDataReader.java
index 214ece3..5282533 100644
--- a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FSInputGeneralColumnDataReader.java
+++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/FSInputGeneralColumnDataReader.java
@@ -20,6 +20,7 @@ package org.apache.kylin.stream.core.storage.columnar;
import java.io.IOException;
import java.util.Iterator;
+import java.util.NoSuchElementException;
import org.apache.hadoop.fs.FSDataInputStream;
@@ -59,7 +60,7 @@ public class FSInputGeneralColumnDataReader implements ColumnDataReader {
readRowCount++;
return result;
} catch (IOException e) {
- throw new RuntimeException("error when read data", e);
+ throw new NoSuchElementException("error when read data");
}
}
diff --git a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputLZ4CompressedColumnReader.java b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputLZ4CompressedColumnReader.java
index f9e4560..0de561e 100644
--- a/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputLZ4CompressedColumnReader.java
+++ b/stream-core/src/main/java/org/apache/kylin/stream/core/storage/columnar/compress/FSInputLZ4CompressedColumnReader.java
@@ -21,13 +21,14 @@ package org.apache.kylin.stream.core.storage.columnar.compress;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Iterator;
-
-import net.jpountz.lz4.LZ4Factory;
-import net.jpountz.lz4.LZ4SafeDecompressor;
+import java.util.NoSuchElementException;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.kylin.stream.core.storage.columnar.ColumnDataReader;
+import net.jpountz.lz4.LZ4Factory;
+import net.jpountz.lz4.LZ4SafeDecompressor;
+
public class FSInputLZ4CompressedColumnReader implements ColumnDataReader {
private int rowCount;
@@ -89,7 +90,7 @@ public class FSInputLZ4CompressedColumnReader implements ColumnDataReader {
try {
loadNextBuffer();
} catch (IOException e) {
- throw new RuntimeException("error when read data", e);
+ throw new NoSuchElementException("error when read data");
}
}
byte[] readBuffer = new byte[valLen];