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 2017/01/20 10:07:42 UTC
[1/7] kylin git commit: KYLIN-2387 code refactor,
merge ImmutableBitmapCounter and MutableBitmapCounter into
RoaringBitmapCounter [Forced Update!]
Repository: kylin
Updated Branches:
refs/heads/master-hbase1.x aa4e2088f -> 5feb9e928 (forced update)
KYLIN-2387 code refactor, merge ImmutableBitmapCounter and MutableBitmapCounter into RoaringBitmapCounter
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/38c3e7bf
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/38c3e7bf
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/38c3e7bf
Branch: refs/heads/master-hbase1.x
Commit: 38c3e7bf691ecdfd0f8d42fcc97065a0596be018
Parents: 61833d9
Author: gaodayue <ga...@meituan.com>
Authored: Thu Jan 19 15:23:34 2017 +0800
Committer: gaodayue <ga...@meituan.com>
Committed: Thu Jan 19 15:43:14 2017 +0800
----------------------------------------------------------------------
.../gridtable/AggregationCacheMemSizeTest.java | 11 +-
.../metadata/measure/MeasureCodecTest.java | 5 +-
.../kylin/measure/bitmap/BitmapAggregator.java | 25 +---
.../kylin/measure/bitmap/BitmapCounter.java | 28 +++-
.../measure/bitmap/BitmapCounterFactory.java | 30 ++++
.../BitmapIntersectDistinctCountAggFunc.java | 16 +-
.../kylin/measure/bitmap/BitmapMeasureType.java | 17 ++-
.../kylin/measure/bitmap/BitmapSerializer.java | 9 +-
.../measure/bitmap/ImmutableBitmapCounter.java | 108 --------------
.../measure/bitmap/MutableBitmapCounter.java | 60 --------
.../measure/bitmap/RoaringBitmapCounter.java | 147 +++++++++++++++++++
.../bitmap/RoaringBitmapCounterFactory.java | 47 ++++++
.../measure/AggregatorMemEstimateTest.java | 5 +-
.../measure/bitmap/BitmapAggregatorTest.java | 56 ++++++-
.../kylin/measure/bitmap/BitmapCounterTest.java | 32 +---
.../measure/bitmap/BitmapSerializerTest.java | 5 +-
16 files changed, 345 insertions(+), 256 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
index 63c7672..f749fb4 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
@@ -33,7 +33,9 @@ import org.apache.kylin.measure.basic.BigDecimalSumAggregator;
import org.apache.kylin.measure.basic.DoubleSumAggregator;
import org.apache.kylin.measure.basic.LongSumAggregator;
import org.apache.kylin.measure.bitmap.BitmapAggregator;
-import org.apache.kylin.measure.bitmap.MutableBitmapCounter;
+import org.apache.kylin.measure.bitmap.BitmapCounter;
+import org.apache.kylin.measure.bitmap.BitmapCounterFactory;
+import org.apache.kylin.measure.bitmap.RoaringBitmapCounterFactory;
import org.apache.kylin.measure.hllc.HLLCAggregator;
import org.apache.kylin.measure.hllc.HLLCounter;
import org.github.jamm.MemoryMeter;
@@ -43,13 +45,14 @@ import com.google.common.base.Stopwatch;
public class AggregationCacheMemSizeTest {
private static final MemoryMeter meter = new MemoryMeter();
- private static final MutableBitmapCounter[] bitmaps = new MutableBitmapCounter[5];
+ private static final BitmapCounterFactory bitmapFactory = RoaringBitmapCounterFactory.INSTANCE;
+ private static final BitmapCounter[] bitmaps = new BitmapCounter[5];
private static final Random random = new Random();
// consider bitmaps with variant cardinality
static {
for (int i = 0; i < bitmaps.length; i++) {
- bitmaps[i] = new MutableBitmapCounter();
+ bitmaps[i] = bitmapFactory.newBitmap();
}
final int totalBits = 1_000_000;
@@ -116,7 +119,7 @@ public class AggregationCacheMemSizeTest {
}
private BitmapAggregator createBitmapAggr(boolean lowCardinality) {
- MutableBitmapCounter counter = new MutableBitmapCounter();
+ BitmapCounter counter = bitmapFactory.newBitmap();
counter.orWith(lowCardinality ? bitmaps[0] : bitmaps[3]);
BitmapAggregator result = new BitmapAggregator();
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java b/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
index 97c9751..7129a5e 100644
--- a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
@@ -25,7 +25,8 @@ import java.nio.ByteBuffer;
import org.apache.kylin.common.util.LocalFileMetadataTestCase;
import org.apache.kylin.measure.BufferedMeasureCodec;
-import org.apache.kylin.measure.bitmap.MutableBitmapCounter;
+import org.apache.kylin.measure.bitmap.BitmapCounter;
+import org.apache.kylin.measure.bitmap.RoaringBitmapCounterFactory;
import org.apache.kylin.measure.hllc.HLLCounter;
import org.apache.kylin.metadata.model.FunctionDesc;
import org.apache.kylin.metadata.model.MeasureDesc;
@@ -58,7 +59,7 @@ public class MeasureCodecTest extends LocalFileMetadataTestCase {
HLLCounter hllc = new HLLCounter(16);
hllc.add("1234567");
hllc.add("abcdefg");
- MutableBitmapCounter bitmap = new MutableBitmapCounter();
+ BitmapCounter bitmap = RoaringBitmapCounterFactory.INSTANCE.newBitmap();
bitmap.add(123);
bitmap.add(45678);
bitmap.add(Integer.MAX_VALUE - 10);
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java
index 2c91bfa..d57af48 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java
@@ -21,47 +21,36 @@ package org.apache.kylin.measure.bitmap;
import org.apache.kylin.measure.MeasureAggregator;
public class BitmapAggregator extends MeasureAggregator<BitmapCounter> {
+ private static final BitmapCounterFactory bitmapFactory = RoaringBitmapCounterFactory.INSTANCE;
- private ImmutableBitmapCounter sum;
- private boolean isMutable;
+ private BitmapCounter sum;
@Override
public void reset() {
sum = null;
- isMutable = false;
}
@Override
public void aggregate(BitmapCounter value) {
- ImmutableBitmapCounter v = (ImmutableBitmapCounter) value;
-
// Here we optimize for case when group only has 1 value. In such situation, no
// aggregation is needed, so we just keep a reference to the first value, saving
// the cost of deserialization and merging.
if (sum == null) {
- sum = v;
+ sum = value;
return;
}
- MutableBitmapCounter mutable;
- if (!isMutable) { // when aggregate the second value
- mutable = sum.toMutable();
- sum = mutable;
- isMutable = true;
- } else { // for the third, forth, ...
- mutable = (MutableBitmapCounter) sum;
- }
- mutable.orWith(v);
+ sum.orWith(value);
}
@Override
public BitmapCounter aggregate(BitmapCounter value1, BitmapCounter value2) {
- MutableBitmapCounter merged = new MutableBitmapCounter();
+ BitmapCounter merged = bitmapFactory.newBitmap();
if (value1 != null) {
- merged.orWith((ImmutableBitmapCounter) value1);
+ merged.orWith(value1);
}
if (value2 != null) {
- merged.orWith((ImmutableBitmapCounter) value2);
+ merged.orWith(value2);
}
return merged;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
index f07059c..0854b6d 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
@@ -26,6 +26,30 @@ import java.util.Iterator;
* An implementation-agnostic bitmap type.
*/
public interface BitmapCounter extends Iterable<Integer> {
+
+ /**
+ * Add the value to the bitmap (set the value to "true"), whether it already appears or not.
+ * @param value integer value
+ */
+ void add(int value);
+
+ /**
+ * In-place bitwise OR (union) operation. The current bitmap is modified.
+ * @param another other bitmap
+ */
+ void orWith(BitmapCounter another);
+
+ /**
+ * In-place bitwise AND (intersection) operation. The current bitmap is modified.
+ * @param another other bitmap
+ */
+ void andWith(BitmapCounter another);
+
+ /**
+ * reset to an empty bitmap
+ */
+ void clear();
+
/**
* @return cardinality of the bitmap
*/
@@ -44,13 +68,13 @@ public interface BitmapCounter extends Iterable<Integer> {
/**
* Serialize this counter. The current counter is not modified.
*/
- void serialize(ByteBuffer out) throws IOException;
+ void write(ByteBuffer out) throws IOException;
/**
* Deserialize a counter from its serialized form.
* <p> After deserialize, any changes to `in` should not affect the returned counter.
*/
- BitmapCounter deserialize(ByteBuffer in) throws IOException;
+ void readFields(ByteBuffer in) throws IOException;
/**
* @return size of the counter stored in the current position of `in`.
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounterFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounterFactory.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounterFactory.java
new file mode 100644
index 0000000..da7748e
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounterFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.measure.bitmap;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public interface BitmapCounterFactory {
+ BitmapCounter newBitmap();
+
+ BitmapCounter newBitmap(int... values);
+
+ BitmapCounter newBitmap(ByteBuffer in) throws IOException;
+}
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapIntersectDistinctCountAggFunc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapIntersectDistinctCountAggFunc.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapIntersectDistinctCountAggFunc.java
index dcdf945..cd4d306 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapIntersectDistinctCountAggFunc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapIntersectDistinctCountAggFunc.java
@@ -28,9 +28,10 @@ import java.util.Map;
* requires an bitmap count distinct measure of uuid, and an dimension of event
*/
public class BitmapIntersectDistinctCountAggFunc {
+ private static final BitmapCounterFactory factory = RoaringBitmapCounterFactory.INSTANCE;
public static class RetentionPartialResult {
- Map<Object, MutableBitmapCounter> map;
+ Map<Object, BitmapCounter> map;
List keyList;
public RetentionPartialResult() {
@@ -42,12 +43,11 @@ public class BitmapIntersectDistinctCountAggFunc {
this.keyList = keyList;
}
if (this.keyList != null && this.keyList.contains(key)) {
- MutableBitmapCounter counter = map.get(key);
+ BitmapCounter counter = map.get(key);
if (counter == null) {
- counter = new MutableBitmapCounter();
- map.put(key, counter);
+ map.put(key, counter = factory.newBitmap());
}
- counter.orWith((ImmutableBitmapCounter) value);
+ counter.orWith((BitmapCounter) value);
}
}
@@ -61,11 +61,11 @@ public class BitmapIntersectDistinctCountAggFunc {
return 0;
}
}
- MutableBitmapCounter counter = null;
+ BitmapCounter counter = null;
for (Object key : keyList) {
- MutableBitmapCounter c = map.get(key);
+ BitmapCounter c = map.get(key);
if (counter == null) {
- counter = new MutableBitmapCounter();
+ counter = factory.newBitmap();
counter.orWith(c);
} else {
counter.andWith(c);
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
index de2a34a..e4fb079 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
@@ -96,18 +96,19 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
@Override
public MeasureIngester<BitmapCounter> newIngester() {
+ final BitmapCounterFactory factory = RoaringBitmapCounterFactory.INSTANCE;
+
return new MeasureIngester<BitmapCounter>() {
- MutableBitmapCounter current = new MutableBitmapCounter();
+ BitmapCounter current = factory.newBitmap();
@Override
public BitmapCounter valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
checkArgument(values.length == 1, "expect 1 value, got %s", Arrays.toString(values));
- MutableBitmapCounter bitmap = current;
- bitmap.clear();
+ current.clear();
if (values[0] == null) {
- return bitmap;
+ return current;
}
int id;
@@ -119,8 +120,8 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
id = Integer.parseInt(values[0]);
}
- bitmap.add(id);
- return bitmap;
+ current.add(id);
+ return current;
}
@Override
@@ -132,7 +133,7 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
Dictionary<String> sourceDict = oldDicts.get(colRef);
Dictionary<String> mergedDict = newDicts.get(colRef);
- MutableBitmapCounter retValue = new MutableBitmapCounter();
+ BitmapCounter retValue = factory.newBitmap();
for (int id : value) {
int newId;
String v = sourceDict.getValueFromId(id);
@@ -148,7 +149,7 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
@Override
public void reset() {
- current = new MutableBitmapCounter();
+ current = factory.newBitmap();
}
};
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
index 0e970de..c1b260d 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
@@ -25,7 +25,8 @@ import java.io.IOException;
import java.nio.ByteBuffer;
public class BitmapSerializer extends DataTypeSerializer<BitmapCounter> {
- private static final BitmapCounter DELEGATE = new MutableBitmapCounter();
+ private static final BitmapCounterFactory factory = RoaringBitmapCounterFactory.INSTANCE;
+ private static final BitmapCounter DELEGATE = factory.newBitmap();
// called by reflection
public BitmapSerializer(DataType type) {
@@ -34,7 +35,7 @@ public class BitmapSerializer extends DataTypeSerializer<BitmapCounter> {
@Override
public void serialize(BitmapCounter value, ByteBuffer out) {
try {
- value.serialize(out);
+ value.write(out);
} catch (IOException e) {
throw new RuntimeException(e);
}
@@ -42,9 +43,9 @@ public class BitmapSerializer extends DataTypeSerializer<BitmapCounter> {
@Override
public BitmapCounter deserialize(ByteBuffer in) {
-
try {
- return DELEGATE.deserialize(in);
+ return factory.newBitmap(in);
+
} catch (IOException e) {
throw new RuntimeException(e);
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/ImmutableBitmapCounter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/ImmutableBitmapCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/ImmutableBitmapCounter.java
deleted file mode 100644
index 753f089..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/ImmutableBitmapCounter.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.measure.bitmap;
-
-import org.apache.kylin.common.util.ByteBufferOutputStream;
-import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.BufferOverflowException;
-import java.nio.ByteBuffer;
-import java.util.Iterator;
-
-/**
- * A thin wrapper around {@link ImmutableRoaringBitmap}.
- */
-public class ImmutableBitmapCounter implements BitmapCounter {
-
- protected ImmutableRoaringBitmap bitmap;
-
- public ImmutableBitmapCounter(ImmutableRoaringBitmap bitmap) {
- this.bitmap = bitmap;
- }
-
- @Override
- public long getCount() {
- return bitmap.getCardinality();
- }
-
- @Override
- public int getMemBytes() {
- return bitmap.getSizeInBytes();
- }
-
- @Override
- public Iterator<Integer> iterator() {
- return bitmap.iterator();
- }
-
- @Override
- public void serialize(ByteBuffer out) throws IOException {
- if (out.remaining() < bitmap.serializedSizeInBytes()) {
- throw new BufferOverflowException();
- }
- bitmap.serialize(new DataOutputStream(new ByteBufferOutputStream(out)));
- }
-
- @Override
- public BitmapCounter deserialize(ByteBuffer in) throws IOException {
- int size = peekLength(in);
- // make a copy of the content to be safe
- byte[] dst = new byte[size];
- in.get(dst);
-
- // just map the buffer, faster than deserialize
- ImmutableRoaringBitmap bitmap = new ImmutableRoaringBitmap(ByteBuffer.wrap(dst));
- return new ImmutableBitmapCounter(bitmap);
- }
-
- @Override
- public int peekLength(ByteBuffer in) {
- // only look at the metadata of the bitmap, no deserialization happens
- ImmutableRoaringBitmap bitmap = new ImmutableRoaringBitmap(in);
- return bitmap.serializedSizeInBytes();
- }
-
- /**
- * Copies the content of this counter to a counter that can be modified.
- * @return a mutable counter
- */
- public MutableBitmapCounter toMutable() {
- MutableBitmapCounter mutable = new MutableBitmapCounter();
- mutable.orWith(this);
- return mutable;
- }
-
- @Override
- public boolean equals(Object obj) {
- return (obj instanceof ImmutableBitmapCounter) &&
- bitmap.equals(((ImmutableBitmapCounter) obj).bitmap);
- }
-
- @Override
- public int hashCode() {
- return bitmap.hashCode();
- }
-
- @Override
- public String toString() {
- return "BitmapCounter[" + getCount() + "]";
- }
-}
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/MutableBitmapCounter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/MutableBitmapCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/MutableBitmapCounter.java
deleted file mode 100644
index af01790..0000000
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/MutableBitmapCounter.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.kylin.measure.bitmap;
-
-import org.roaringbitmap.buffer.MutableRoaringBitmap;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-/**
- * A thin wrapper around {@link MutableRoaringBitmap}.
- */
-public class MutableBitmapCounter extends ImmutableBitmapCounter {
-
- public MutableBitmapCounter() {
- super(new MutableRoaringBitmap());
- }
-
- private MutableRoaringBitmap getBitmap() {
- return (MutableRoaringBitmap) bitmap;
- }
-
- public void clear() {
- getBitmap().clear();
- }
-
- public void add(int value) {
- getBitmap().add(value);
- }
-
- public void orWith(ImmutableBitmapCounter another) {
- getBitmap().or(another.bitmap);
- }
-
- public void andWith(ImmutableBitmapCounter another) {
- getBitmap().and(another.bitmap);
- }
-
- @Override
- public void serialize(ByteBuffer out) throws IOException {
- getBitmap().runOptimize();
- super.serialize(out);
- }
-}
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounter.java
new file mode 100644
index 0000000..cd07d20
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounter.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.measure.bitmap;
+
+import org.apache.kylin.common.util.ByteBufferOutputStream;
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
+import org.roaringbitmap.buffer.MutableRoaringBitmap;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+
+/**
+ * A {@link BitmapCounter} based on roaring bitmap.
+ */
+public class RoaringBitmapCounter implements BitmapCounter {
+
+ private ImmutableRoaringBitmap bitmap;
+
+ RoaringBitmapCounter() {
+ bitmap = new MutableRoaringBitmap();
+ }
+
+ RoaringBitmapCounter(ImmutableRoaringBitmap bitmap) {
+ this.bitmap = bitmap;
+ }
+
+ private MutableRoaringBitmap getMutableBitmap() {
+ if (bitmap instanceof MutableRoaringBitmap) {
+ return (MutableRoaringBitmap) bitmap;
+ }
+ // convert to mutable bitmap
+ MutableRoaringBitmap result = bitmap.toMutableRoaringBitmap();
+ bitmap = result;
+ return result;
+ }
+
+ @Override
+ public void add(int value) {
+ getMutableBitmap().add(value);
+ }
+
+ @Override
+ public void orWith(BitmapCounter another) {
+ if (another instanceof RoaringBitmapCounter) {
+ RoaringBitmapCounter input = (RoaringBitmapCounter) another;
+ getMutableBitmap().or(input.bitmap);
+ return;
+ }
+ throw new IllegalArgumentException("Unsupported type: " + another.getClass().getCanonicalName());
+ }
+
+ @Override
+ public void andWith(BitmapCounter another) {
+ if (another instanceof RoaringBitmapCounter) {
+ RoaringBitmapCounter input = (RoaringBitmapCounter) another;
+ getMutableBitmap().and(input.bitmap);
+ return;
+ }
+ throw new IllegalArgumentException("Unsupported type: " + another.getClass().getCanonicalName());
+ }
+
+ @Override
+ public void clear() {
+ bitmap = new MutableRoaringBitmap();
+ }
+
+ @Override
+ public long getCount() {
+ return bitmap.getCardinality();
+ }
+
+ @Override
+ public int getMemBytes() {
+ return bitmap.getSizeInBytes();
+ }
+
+ @Override
+ public Iterator<Integer> iterator() {
+ return bitmap.iterator();
+ }
+
+ @Override
+ public void write(ByteBuffer out) throws IOException {
+ if (bitmap instanceof MutableRoaringBitmap) {
+ getMutableBitmap().runOptimize();
+ }
+
+ if (out.remaining() < bitmap.serializedSizeInBytes()) {
+ throw new BufferOverflowException();
+ }
+ bitmap.serialize(new DataOutputStream(new ByteBufferOutputStream(out)));
+ }
+
+ @Override
+ public void readFields(ByteBuffer in) throws IOException {
+ int size = peekLength(in);
+ // make a copy of the content to be safe
+ byte[] dst = new byte[size];
+ in.get(dst);
+
+ // ImmutableRoaringBitmap only maps the buffer, thus faster than constructing a MutableRoaringBitmap.
+ // we'll convert to MutableRoaringBitmap later when mutate is needed
+ bitmap = new ImmutableRoaringBitmap(ByteBuffer.wrap(dst));
+ }
+
+ @Override
+ public int peekLength(ByteBuffer in) {
+ // only look at the metadata of the bitmap, no deserialization happens
+ ImmutableRoaringBitmap bitmap = new ImmutableRoaringBitmap(in);
+ return bitmap.serializedSizeInBytes();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ return (obj instanceof RoaringBitmapCounter) &&
+ bitmap.equals(((RoaringBitmapCounter) obj).bitmap);
+ }
+
+ @Override
+ public int hashCode() {
+ return bitmap.hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return "RoaringBitmapCounter[" + getCount() + "]";
+ }
+}
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounterFactory.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounterFactory.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounterFactory.java
new file mode 100644
index 0000000..a71df95
--- /dev/null
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/RoaringBitmapCounterFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kylin.measure.bitmap;
+
+import org.roaringbitmap.buffer.MutableRoaringBitmap;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class RoaringBitmapCounterFactory implements BitmapCounterFactory {
+ public static final BitmapCounterFactory INSTANCE = new RoaringBitmapCounterFactory();
+
+ private RoaringBitmapCounterFactory() {}
+
+ @Override
+ public BitmapCounter newBitmap() {
+ return new RoaringBitmapCounter();
+ }
+
+ @Override
+ public BitmapCounter newBitmap(int... values) {
+ return new RoaringBitmapCounter(MutableRoaringBitmap.bitmapOf(values));
+ }
+
+ @Override
+ public BitmapCounter newBitmap(ByteBuffer in) throws IOException {
+ RoaringBitmapCounter counter = new RoaringBitmapCounter();
+ counter.readFields(in);
+ return counter;
+ }
+}
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java
index 39921c2..4e67d22 100644
--- a/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java
@@ -33,7 +33,8 @@ import org.apache.kylin.measure.basic.LongMaxAggregator;
import org.apache.kylin.measure.basic.LongMinAggregator;
import org.apache.kylin.measure.basic.LongSumAggregator;
import org.apache.kylin.measure.bitmap.BitmapAggregator;
-import org.apache.kylin.measure.bitmap.MutableBitmapCounter;
+import org.apache.kylin.measure.bitmap.BitmapCounter;
+import org.apache.kylin.measure.bitmap.RoaringBitmapCounterFactory;
import org.apache.kylin.measure.extendedcolumn.ExtendedColumnMeasureType;
import org.apache.kylin.measure.hllc.HLLCAggregator;
import org.apache.kylin.measure.hllc.HLLCounter;
@@ -104,7 +105,7 @@ public class AggregatorMemEstimateTest extends LocalFileMetadataTestCase {
hllcAggregator.aggregate(new HLLCounter(14));
BitmapAggregator bitmapAggregator = new BitmapAggregator();
- MutableBitmapCounter bitmapCounter = new MutableBitmapCounter();
+ BitmapCounter bitmapCounter = RoaringBitmapCounterFactory.INSTANCE.newBitmap();
for (int i = 4000; i <= 100000; i += 2) {
bitmapCounter.add(i);
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapAggregatorTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapAggregatorTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapAggregatorTest.java
index a52e427..0b82fc4 100644
--- a/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapAggregatorTest.java
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapAggregatorTest.java
@@ -19,30 +19,72 @@
package org.apache.kylin.measure.bitmap;
import org.junit.Test;
-import org.roaringbitmap.buffer.MutableRoaringBitmap;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
public class BitmapAggregatorTest {
+ private static final BitmapCounterFactory factory = RoaringBitmapCounterFactory.INSTANCE;
@Test
public void testAggregator() {
BitmapAggregator aggregator = new BitmapAggregator();
assertNull(null, aggregator.getState());
- aggregator.aggregate(new ImmutableBitmapCounter(
- MutableRoaringBitmap.bitmapOf(10, 20, 30, 40)
- ));
+ aggregator.aggregate(factory.newBitmap(10, 20, 30, 40));
assertEquals(4, aggregator.getState().getCount());
- aggregator.aggregate(new ImmutableBitmapCounter(
- MutableRoaringBitmap.bitmapOf(25, 30, 35, 40, 45)
- ));
+ aggregator.aggregate(factory.newBitmap(25, 30, 35, 40, 45));
assertEquals(7, aggregator.getState().getCount());
aggregator.reset();
assertNull(aggregator.getState());
}
+ @Test
+ public void testAggregatorDeserializedCounter() throws IOException {
+ BitmapCounter counter1 = factory.newBitmap(1, 3, 5);
+ BitmapCounter counter2 = factory.newBitmap(1, 2, 4, 6);
+ BitmapCounter counter3 = factory.newBitmap(1, 5, 7);
+
+ ByteBuffer buffer = ByteBuffer.allocate(1024 * 1024);
+ counter1.write(buffer);
+ counter2.write(buffer);
+ counter3.write(buffer);
+ buffer.flip();
+
+ BitmapAggregator aggregator = new BitmapAggregator();
+
+ // first
+ BitmapCounter next = factory.newBitmap(buffer);
+ assertEquals(counter1, next);
+
+ aggregator.aggregate(next);
+ assertEquals(counter1, aggregator.getState());
+
+ // second
+ next = factory.newBitmap(buffer);
+ assertEquals(counter2, next);
+
+ aggregator.aggregate(next);
+ assertEquals(6, aggregator.getState().getCount());
+
+ // third
+ next = factory.newBitmap(buffer);
+ assertEquals(counter3, next);
+
+ aggregator.aggregate(next);
+ assertEquals(7, aggregator.getState().getCount());
+
+ BitmapCounter result = factory.newBitmap();
+ result.orWith(counter1);
+ result.orWith(counter2);
+ result.orWith(counter3);
+ assertEquals(result, aggregator.getState());
+
+
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapCounterTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapCounterTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapCounterTest.java
index ee7733f..7194a23 100644
--- a/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapCounterTest.java
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapCounterTest.java
@@ -19,23 +19,21 @@
package org.apache.kylin.measure.bitmap;
import org.junit.Test;
-import org.roaringbitmap.buffer.MutableRoaringBitmap;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
public class BitmapCounterTest {
+ private static final BitmapCounterFactory factory = RoaringBitmapCounterFactory.INSTANCE;
@Test
public void testBitmapCounter() {
- ImmutableBitmapCounter counter = new ImmutableBitmapCounter(
- MutableRoaringBitmap.bitmapOf(10, 20, 30, 1000)
- );
+ BitmapCounter counter = factory.newBitmap(10, 20, 30, 1000);
assertEquals(4, counter.getCount());
assertTrue(counter.getMemBytes() > 0);
- MutableBitmapCounter counter2 = new MutableBitmapCounter();
+ BitmapCounter counter2 = factory.newBitmap();
assertEquals(0, counter2.getCount());
counter2.add(10);
counter2.add(30);
@@ -58,28 +56,4 @@ public class BitmapCounterTest {
assertEquals(0, counter2.getCount());
}
- @Test
- public void testToMutableBitmapCounter() {
- ImmutableBitmapCounter immutable = new ImmutableBitmapCounter(
- MutableRoaringBitmap.bitmapOf(10, 20, 30, 1000)
- );
- MutableBitmapCounter mutable = new MutableBitmapCounter();
- mutable.orWith(immutable);
-
- assertEquals(4, immutable.getCount());
- assertEquals(4, mutable.getCount());
- assertTrue(immutable.equals(mutable));
- assertTrue(mutable.equals(immutable));
-
- MutableBitmapCounter newCounter = immutable.toMutable();
- newCounter.add(40);
- assertEquals(4, immutable.getCount());
- assertEquals(5, newCounter.getCount());
-
- newCounter = mutable.toMutable();
- newCounter.add(40);
- assertEquals(4, mutable.getCount());
- assertEquals(5, newCounter.getCount());
- }
-
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kylin/blob/38c3e7bf/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapSerializerTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapSerializerTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapSerializerTest.java
index e5b828d..acbfe88 100644
--- a/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapSerializerTest.java
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/bitmap/BitmapSerializerTest.java
@@ -24,7 +24,6 @@ import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
-import org.roaringbitmap.buffer.MutableRoaringBitmap;
import java.nio.BufferOverflowException;
import java.nio.ByteBuffer;
@@ -47,9 +46,7 @@ public class BitmapSerializerTest extends LocalFileMetadataTestCase {
public void testBitmapSerDe() {
BitmapSerializer serializer = new BitmapSerializer(DataType.ANY);
- ImmutableBitmapCounter counter = new ImmutableBitmapCounter(
- MutableRoaringBitmap.bitmapOf(1, 1234, 5678, 100000)
- );
+ BitmapCounter counter = RoaringBitmapCounterFactory.INSTANCE.newBitmap(1, 1234, 5678, 100000);
ByteBuffer buffer = ByteBuffer.allocate(1024 * 1024);
serializer.serialize(counter, buffer);
[6/7] kylin git commit: KYLIN-1528 Create a branch for v1.5 with
HBase 1.x API
Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
index c8410f9..e72859d 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/DeployCoprocessorCLI.java
@@ -44,7 +44,8 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.KylinVersion;
@@ -81,7 +82,8 @@ public class DeployCoprocessorCLI {
KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
FileSystem fileSystem = FileSystem.get(hconf);
- HBaseAdmin hbaseAdmin = new HBaseAdmin(hconf);
+ Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
+ Admin hbaseAdmin = conn.getAdmin();
String localCoprocessorJar;
if ("default".equals(args[0])) {
@@ -165,10 +167,10 @@ public class DeployCoprocessorCLI {
public static void deployCoprocessor(HTableDescriptor tableDesc) {
try {
initHTableCoprocessor(tableDesc);
- logger.info("hbase table " + tableDesc.getName() + " deployed with coprocessor.");
+ logger.info("hbase table " + tableDesc.getTableName() + " deployed with coprocessor.");
} catch (Exception ex) {
- logger.error("Error deploying coprocessor on " + tableDesc.getName(), ex);
+ logger.error("Error deploying coprocessor on " + tableDesc.getTableName(), ex);
logger.error("Will try creating the table without coprocessor.");
}
}
@@ -189,7 +191,7 @@ public class DeployCoprocessorCLI {
desc.addCoprocessor(CubeEndpointClass, hdfsCoprocessorJar, 1001, null);
}
- public static boolean resetCoprocessor(String tableName, HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
+ public static boolean resetCoprocessor(String tableName, Admin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
@@ -204,7 +206,7 @@ public class DeployCoprocessorCLI {
logger.info("reset coprocessor on " + tableName);
logger.info("Disable " + tableName);
- hbaseAdmin.disableTable(tableName);
+ hbaseAdmin.disableTable(TableName.valueOf(tableName));
while (desc.hasCoprocessor(CubeObserverClassOld2)) {
desc.removeCoprocessor(CubeObserverClassOld2);
@@ -230,16 +232,15 @@ public class DeployCoprocessorCLI {
desc.setValue(IRealizationConstants.HTableGitTag, commitInfo);
}
- hbaseAdmin.modifyTable(tableName, desc);
+ hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
logger.info("Enable " + tableName);
- hbaseAdmin.enableTable(tableName);
+ hbaseAdmin.enableTable(TableName.valueOf(tableName));
return true;
}
-
- private static List<String> resetCoprocessorOnHTables(final HBaseAdmin hbaseAdmin, final Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
+ private static List<String> resetCoprocessorOnHTables(final Admin hbaseAdmin, final Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
List<String> processedTables = Collections.synchronizedList(new ArrayList<String>());
ExecutorService coprocessorPool = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors() * 2);
CountDownLatch countDownLatch = new CountDownLatch(tableNames.size());
@@ -260,12 +261,12 @@ public class DeployCoprocessorCLI {
private static class ResetCoprocessorWorker implements Runnable {
private final CountDownLatch countDownLatch;
- private final HBaseAdmin hbaseAdmin;
+ private final Admin hbaseAdmin;
private final Path hdfsCoprocessorJar;
private final String tableName;
private final List<String> processedTables;
- public ResetCoprocessorWorker(CountDownLatch countDownLatch, HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar, String tableName, List<String> processedTables) {
+ public ResetCoprocessorWorker(CountDownLatch countDownLatch, Admin hbaseAdmin, Path hdfsCoprocessorJar, String tableName, List<String> processedTables) {
this.countDownLatch = countDownLatch;
this.hbaseAdmin = hbaseAdmin;
this.hdfsCoprocessorJar = hdfsCoprocessorJar;
@@ -386,7 +387,7 @@ public class DeployCoprocessorCLI {
return coprocessorDir;
}
- private static Set<String> getCoprocessorJarPaths(HBaseAdmin hbaseAdmin, List<String> tableNames) throws IOException {
+ private static Set<String> getCoprocessorJarPaths(Admin hbaseAdmin, List<String> tableNames) throws IOException {
HashSet<String> result = new HashSet<String>();
for (String tableName : tableNames) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
index 61c73d5..1cdb2f8 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
@@ -25,10 +25,11 @@ import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.persistence.JsonSerializer;
import org.apache.kylin.common.persistence.ResourceStore;
@@ -235,9 +236,9 @@ public class ExtendCubeToHybridCLI {
Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
ProjectInstance project = store.getResource(projectResPath, ProjectInstance.class, projectSerializer);
String projUUID = project.getUuid();
- HTableInterface aclHtable = null;
+ Table aclHtable = null;
try {
- aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(kylinConfig.getMetadataUrlPrefix() + "_acl");
+ aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(TableName.valueOf(kylinConfig.getMetadataUrlPrefix() + "_acl"));
// cube acl
Result result = aclHtable.get(new Get(Bytes.toBytes(origCubeId)));
@@ -257,7 +258,6 @@ public class ExtendCubeToHybridCLI {
aclHtable.put(put);
}
}
- aclHtable.flushCommits();
} finally {
IOUtils.closeQuietly(aclHtable);
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
index 86ba22f..dd5f8fa 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/GridTableHBaseBenchmark.java
@@ -28,13 +28,13 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
import org.apache.kylin.common.util.Bytes;
import org.apache.kylin.common.util.Pair;
@@ -75,7 +75,7 @@ public class GridTableHBaseBenchmark {
System.out.println("Testing grid table scanning, hit ratio " + hitRatio + ", index ratio " + indexRatio);
String hbaseUrl = "hbase"; // use hbase-site.xml on classpath
- HConnection conn = HBaseConnection.get(hbaseUrl);
+ Connection conn = HBaseConnection.get(hbaseUrl);
createHTableIfNeeded(conn, TEST_TABLE);
prepareData(conn);
@@ -91,10 +91,10 @@ public class GridTableHBaseBenchmark {
}
- private static void testColumnScan(HConnection conn, List<Pair<Integer, Integer>> colScans) throws IOException {
+ private static void testColumnScan(Connection conn, List<Pair<Integer, Integer>> colScans) throws IOException {
Stats stats = new Stats("COLUMN_SCAN");
- HTableInterface table = conn.getTable(TEST_TABLE);
+ Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
try {
stats.markStart();
@@ -122,20 +122,20 @@ public class GridTableHBaseBenchmark {
}
}
- private static void testRowScanNoIndexFullScan(HConnection conn, boolean[] hits) throws IOException {
+ private static void testRowScanNoIndexFullScan(Connection conn, boolean[] hits) throws IOException {
fullScan(conn, hits, new Stats("ROW_SCAN_NO_IDX_FULL"));
}
- private static void testRowScanNoIndexSkipScan(HConnection conn, boolean[] hits) throws IOException {
+ private static void testRowScanNoIndexSkipScan(Connection conn, boolean[] hits) throws IOException {
jumpScan(conn, hits, new Stats("ROW_SCAN_NO_IDX_SKIP"));
}
- private static void testRowScanWithIndex(HConnection conn, boolean[] hits) throws IOException {
+ private static void testRowScanWithIndex(Connection conn, boolean[] hits) throws IOException {
jumpScan(conn, hits, new Stats("ROW_SCAN_IDX"));
}
- private static void fullScan(HConnection conn, boolean[] hits, Stats stats) throws IOException {
- HTableInterface table = conn.getTable(TEST_TABLE);
+ private static void fullScan(Connection conn, boolean[] hits, Stats stats) throws IOException {
+ Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
try {
stats.markStart();
@@ -156,11 +156,11 @@ public class GridTableHBaseBenchmark {
}
}
- private static void jumpScan(HConnection conn, boolean[] hits, Stats stats) throws IOException {
+ private static void jumpScan(Connection conn, boolean[] hits, Stats stats) throws IOException {
final int jumpThreshold = 6; // compensate for Scan() overhead, totally by experience
- HTableInterface table = conn.getTable(TEST_TABLE);
+ Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
try {
stats.markStart();
@@ -204,8 +204,8 @@ public class GridTableHBaseBenchmark {
}
}
- private static void prepareData(HConnection conn) throws IOException {
- HTableInterface table = conn.getTable(TEST_TABLE);
+ private static void prepareData(Connection conn) throws IOException {
+ Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
try {
// check how many rows existing
@@ -258,8 +258,8 @@ public class GridTableHBaseBenchmark {
return bytes;
}
- private static void createHTableIfNeeded(HConnection conn, String tableName) throws IOException {
- HBaseAdmin hbase = new HBaseAdmin(conn);
+ private static void createHTableIfNeeded(Connection conn, String tableName) throws IOException {
+ Admin hbase = conn.getAdmin();
try {
boolean tableExist = false;
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
index 6749d6c..940d64a 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseClean.java
@@ -24,9 +24,11 @@ import java.util.List;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.OptionBuilder;
import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.AbstractApplication;
import org.apache.kylin.common.util.OptionsHelper;
import org.apache.kylin.metadata.realization.IRealizationConstants;
@@ -55,8 +57,8 @@ public class HBaseClean extends AbstractApplication {
private void cleanUp() {
try {
// get all kylin hbase tables
- Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
- HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+ Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+ Admin hbaseAdmin = conn.getAdmin();
String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
List<String> allTablesNeedToBeDropped = Lists.newArrayList();
@@ -71,12 +73,12 @@ public class HBaseClean extends AbstractApplication {
// drop tables
for (String htableName : allTablesNeedToBeDropped) {
logger.info("Deleting HBase table " + htableName);
- if (hbaseAdmin.tableExists(htableName)) {
- if (hbaseAdmin.isTableEnabled(htableName)) {
- hbaseAdmin.disableTable(htableName);
+ if (hbaseAdmin.tableExists(TableName.valueOf(htableName))) {
+ if (hbaseAdmin.isTableEnabled(TableName.valueOf(htableName))) {
+ hbaseAdmin.disableTable(TableName.valueOf(htableName));
}
- hbaseAdmin.deleteTable(htableName);
+ hbaseAdmin.deleteTable(TableName.valueOf(htableName));
logger.info("Deleted HBase table " + htableName);
} else {
logger.info("HBase table" + htableName + " does not exist");
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
index 937b65f..1daca0a 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseRegionSizeCalculator.java
@@ -23,6 +23,7 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
+import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
@@ -31,12 +32,15 @@ import java.util.TreeSet;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.kylin.common.util.Pair;
import org.slf4j.Logger;
@@ -58,30 +62,31 @@ public class HBaseRegionSizeCalculator {
/**
* Computes size of each region for table and given column families.
* */
- public HBaseRegionSizeCalculator(HTable table) throws IOException {
- this(table, new HBaseAdmin(table.getConfiguration()));
- }
-
- /** Constructor for unit testing */
- HBaseRegionSizeCalculator(HTable table, HBaseAdmin hBaseAdmin) throws IOException {
+ public HBaseRegionSizeCalculator(String tableName, Connection hbaseConnection) throws IOException {
+ Table table = null;
+ Admin admin = null;
try {
+ table = hbaseConnection.getTable(TableName.valueOf(tableName));
+ admin = hbaseConnection.getAdmin();
+
if (!enabled(table.getConfiguration())) {
logger.info("Region size calculation disabled.");
return;
}
- logger.info("Calculating region sizes for table \"" + new String(table.getTableName()) + "\".");
+ logger.info("Calculating region sizes for table \"" + table.getName() + "\".");
// Get regions for table.
- Set<HRegionInfo> tableRegionInfos = table.getRegionLocations().keySet();
+ RegionLocator regionLocator = hbaseConnection.getRegionLocator(table.getName());
+ List<HRegionLocation> regionLocationList = regionLocator.getAllRegionLocations();
Set<byte[]> tableRegions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
- for (HRegionInfo regionInfo : tableRegionInfos) {
- tableRegions.add(regionInfo.getRegionName());
+ for (HRegionLocation hRegionLocation : regionLocationList) {
+ tableRegions.add(hRegionLocation.getRegionInfo().getRegionName());
}
- ClusterStatus clusterStatus = hBaseAdmin.getClusterStatus();
+ ClusterStatus clusterStatus = admin.getClusterStatus();
Collection<ServerName> servers = clusterStatus.getServers();
final long megaByte = 1024L * 1024L;
@@ -105,7 +110,7 @@ public class HBaseRegionSizeCalculator {
}
}
} finally {
- IOUtils.closeQuietly(hBaseAdmin);
+ IOUtils.closeQuietly(admin);
}
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
index 266f7e7..a2f60d4 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HBaseUsage.java
@@ -23,9 +23,10 @@ import java.util.List;
import java.util.Map;
import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.metadata.realization.IRealizationConstants;
import org.apache.kylin.storage.hbase.HBaseConnection;
@@ -42,8 +43,8 @@ public class HBaseUsage {
Map<String, List<String>> envs = Maps.newHashMap();
// get all kylin hbase tables
- Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
- HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+ Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+ Admin hbaseAdmin = conn.getAdmin();
String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
for (HTableDescriptor desc : tableDescriptors) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
index 1db60fb..8dd2164 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HbaseStreamingInput.java
@@ -32,15 +32,15 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
+import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.Bytes;
import org.apache.kylin.storage.hbase.HBaseConnection;
import org.slf4j.Logger;
@@ -58,11 +58,11 @@ public class HbaseStreamingInput {
private static final byte[] QN = "C".getBytes();
public static void createTable(String tableName) throws IOException {
- HConnection conn = getConnection();
- HBaseAdmin hadmin = new HBaseAdmin(conn);
+ Connection conn = getConnection();
+ Admin hadmin = conn.getAdmin();
try {
- boolean tableExist = hadmin.tableExists(tableName);
+ boolean tableExist = hadmin.tableExists(TableName.valueOf(tableName));
if (tableExist) {
logger.info("HTable '" + tableName + "' already exists");
return;
@@ -120,8 +120,8 @@ public class HbaseStreamingInput {
e.printStackTrace();
}
- HConnection conn = getConnection();
- HTableInterface table = conn.getTable(tableName);
+ Connection conn = getConnection();
+ Table table = conn.getTable(TableName.valueOf(tableName));
byte[] key = new byte[8 + 4];//time + id
@@ -136,7 +136,7 @@ public class HbaseStreamingInput {
Bytes.putInt(key, 8, i);
Put put = new Put(key);
byte[] cell = randomBytes(CELL_SIZE);
- put.add(CF, QN, cell);
+ put.addColumn(CF, QN, cell);
buffer.add(put);
}
table.put(buffer);
@@ -172,8 +172,8 @@ public class HbaseStreamingInput {
}
Random r = new Random();
- HConnection conn = getConnection();
- HTableInterface table = conn.getTable(tableName);
+ Connection conn = getConnection();
+ Table table = conn.getTable(TableName.valueOf(tableName));
long leftBound = getFirstKeyTime(table);
long rightBound = System.currentTimeMillis();
@@ -208,7 +208,7 @@ public class HbaseStreamingInput {
}
}
- private static long getFirstKeyTime(HTableInterface table) throws IOException {
+ private static long getFirstKeyTime(Table table) throws IOException {
long startTime = 0;
Scan scan = new Scan();
@@ -226,8 +226,8 @@ public class HbaseStreamingInput {
}
- private static HConnection getConnection() throws IOException {
- return HConnectionManager.createConnection(HBaseConnection.getCurrentHBaseConfiguration());
+ private static Connection getConnection() throws IOException {
+ return HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
}
private static String formatTime(long time) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
index ca1a060..ea05ab2 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/HtableAlterMetadataCLI.java
@@ -23,10 +23,11 @@ import java.io.IOException;
import org.apache.commons.cli.Option;
import org.apache.commons.cli.OptionBuilder;
import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.AbstractApplication;
import org.apache.kylin.common.util.OptionsHelper;
import org.apache.kylin.engine.mr.common.BatchConstants;
@@ -50,8 +51,8 @@ public class HtableAlterMetadataCLI extends AbstractApplication {
String metadataValue;
private void alter() throws IOException {
- Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
- HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+ Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+ Admin hbaseAdmin = conn.getAdmin();
HTableDescriptor table = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
hbaseAdmin.disableTable(table.getTableName());
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
index 8ff5b0f..df4e912 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/OrphanHBaseCleanJob.java
@@ -30,10 +30,14 @@ import org.apache.commons.cli.Options;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.AbstractApplication;
import org.apache.kylin.common.util.OptionsHelper;
import org.apache.kylin.metadata.realization.IRealizationConstants;
+import org.apache.kylin.storage.hbase.HBaseConnection;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -52,9 +56,9 @@ public class OrphanHBaseCleanJob extends AbstractApplication {
Set<String> metastoreWhitelistSet = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
private void cleanUnusedHBaseTables(Configuration conf) throws IOException {
-
+ Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
// get all kylin hbase tables
- HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+ Admin hbaseAdmin = conn.getAdmin();
String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
List<String> allTablesNeedToBeDropped = new ArrayList<String>();
@@ -73,12 +77,13 @@ public class OrphanHBaseCleanJob extends AbstractApplication {
// drop tables
for (String htableName : allTablesNeedToBeDropped) {
logger.info("Deleting HBase table " + htableName);
- if (hbaseAdmin.tableExists(htableName)) {
- if (hbaseAdmin.isTableEnabled(htableName)) {
- hbaseAdmin.disableTable(htableName);
+ TableName tableName = TableName.valueOf(htableName);
+ if (hbaseAdmin.tableExists(tableName)) {
+ if (hbaseAdmin.isTableEnabled(tableName)) {
+ hbaseAdmin.disableTable(tableName);
}
- hbaseAdmin.deleteTable(htableName);
+ hbaseAdmin.deleteTable(tableName);
logger.info("Deleted HBase table " + htableName);
} else {
logger.info("HBase table" + htableName + " does not exist");
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
index 1ea8e8d..bba6745 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/PingHBaseCLI.java
@@ -22,12 +22,13 @@ import java.io.IOException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.token.TokenUtil;
import org.apache.hadoop.security.UserGroupInformation;
@@ -59,12 +60,12 @@ public class PingHBaseCLI {
Scan scan = new Scan();
int limit = 20;
- HConnection conn = null;
- HTableInterface table = null;
+ Connection conn = null;
+ Table table = null;
ResultScanner scanner = null;
try {
- conn = HConnectionManager.createConnection(hconf);
- table = conn.getTable(hbaseTable);
+ conn = ConnectionFactory.createConnection(hconf);
+ table = conn.getTable(TableName.valueOf(hbaseTable));
scanner = table.getScanner(scan);
int count = 0;
for (Result r : scanner) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
index 01edb1f..db516bb 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/RowCounterCLI.java
@@ -22,11 +22,12 @@ import java.io.IOException;
import java.util.Iterator;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.kylin.common.util.Bytes;
import org.apache.kylin.common.util.BytesUtil;
import org.apache.kylin.storage.hbase.HBaseConnection;
@@ -70,8 +71,8 @@ public class RowCounterCLI {
logger.info("My Scan " + scan.toString());
- HConnection conn = HConnectionManager.createConnection(conf);
- HTableInterface tableInterface = conn.getTable(htableName);
+ Connection conn = ConnectionFactory.createConnection(conf);
+ Table tableInterface = conn.getTable(TableName.valueOf(htableName));
Iterator<Result> iterator = tableInterface.getScanner(scan).iterator();
int counter = 0;
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
index 62af2c9..0784305 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/StorageCleanupJob.java
@@ -40,7 +40,9 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.AbstractApplication;
import org.apache.kylin.common.util.CliCommandExecutor;
@@ -57,6 +59,7 @@ import org.apache.kylin.job.execution.AbstractExecutable;
import org.apache.kylin.job.execution.ExecutableManager;
import org.apache.kylin.job.execution.ExecutableState;
import org.apache.kylin.metadata.realization.IRealizationConstants;
+import org.apache.kylin.storage.hbase.HBaseConnection;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -77,7 +80,8 @@ public class StorageCleanupJob extends AbstractApplication {
private void cleanUnusedHBaseTables(Configuration conf) throws IOException {
CubeManager cubeMgr = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
// get all kylin hbase tables
- HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+ Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+ Admin hbaseAdmin = conn.getAdmin();
String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
List<String> allTablesNeedToBeDropped = new ArrayList<String>();
@@ -153,22 +157,22 @@ public class StorageCleanupJob extends AbstractApplication {
}
class DeleteHTableRunnable implements Callable {
- HBaseAdmin hbaseAdmin;
+ Admin hbaseAdmin;
String htableName;
- DeleteHTableRunnable(HBaseAdmin hbaseAdmin, String htableName) {
+ DeleteHTableRunnable(Admin hbaseAdmin, String htableName) {
this.hbaseAdmin = hbaseAdmin;
this.htableName = htableName;
}
public Object call() throws Exception {
logger.info("Deleting HBase table " + htableName);
- if (hbaseAdmin.tableExists(htableName)) {
- if (hbaseAdmin.isTableEnabled(htableName)) {
- hbaseAdmin.disableTable(htableName);
+ if (hbaseAdmin.tableExists(TableName.valueOf(htableName))) {
+ if (hbaseAdmin.isTableEnabled(TableName.valueOf(htableName))) {
+ hbaseAdmin.disableTable(TableName.valueOf(htableName));
}
- hbaseAdmin.deleteTable(htableName);
+ hbaseAdmin.deleteTable(TableName.valueOf(htableName));
logger.info("Deleted HBase table " + htableName);
} else {
logger.info("HBase table" + htableName + " does not exist");
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
index e36f662..42a54c8 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/UpdateHTableHostCLI.java
@@ -24,16 +24,18 @@ import java.util.Arrays;
import java.util.List;
import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.cube.CubeInstance;
import org.apache.kylin.cube.CubeManager;
import org.apache.kylin.cube.CubeSegment;
import org.apache.kylin.metadata.model.SegmentStatusEnum;
import org.apache.kylin.metadata.realization.IRealizationConstants;
-import org.apache.kylin.storage.hbase.HBaseConnection;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -49,14 +51,15 @@ public class UpdateHTableHostCLI {
private List<String> errorMsgs = Lists.newArrayList();
private List<String> htables;
- private HBaseAdmin hbaseAdmin;
+ private Admin hbaseAdmin;
private KylinConfig kylinConfig;
private String oldHostValue;
public UpdateHTableHostCLI(List<String> htables, String oldHostValue) throws IOException {
this.htables = htables;
this.oldHostValue = oldHostValue;
- this.hbaseAdmin = new HBaseAdmin(HBaseConnection.getCurrentHBaseConfiguration());
+ Connection conn = ConnectionFactory.createConnection(HBaseConfiguration.create());
+ hbaseAdmin = conn.getAdmin();
this.kylinConfig = KylinConfig.getInstanceFromEnv();
}
@@ -166,9 +169,9 @@ public class UpdateHTableHostCLI {
HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
if (oldHostValue.equals(desc.getValue(IRealizationConstants.HTableTag))) {
desc.setValue(IRealizationConstants.HTableTag, kylinConfig.getMetadataUrlPrefix());
- hbaseAdmin.disableTable(tableName);
- hbaseAdmin.modifyTable(tableName, desc);
- hbaseAdmin.enableTable(tableName);
+ hbaseAdmin.disableTable(TableName.valueOf(tableName));
+ hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
+ hbaseAdmin.enableTable(TableName.valueOf(tableName));
updatedResources.add(tableName);
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java b/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java
index c8bff89..c0042f3 100644
--- a/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java
+++ b/tool/src/main/java/org/apache/kylin/tool/CubeMigrationCLI.java
@@ -36,9 +36,9 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.persistence.JsonSerializer;
import org.apache.kylin.common.persistence.RawResource;
@@ -231,6 +231,7 @@ public class CubeMigrationCLI {
operations.add(new Opt(OptType.COPY_DICT_OR_SNAPSHOT, new Object[] { item, cube.getName() }));
}
}
+
private static void addCubeAndModelIntoProject(CubeInstance srcCube, String cubeName, String projectName) throws IOException {
String projectResPath = ProjectInstance.concatResourcePath(projectName);
if (!dstStore.exists(projectResPath))
@@ -447,11 +448,11 @@ public class CubeMigrationCLI {
Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
String projUUID = project.getUuid();
- HTableInterface srcAclHtable = null;
- HTableInterface destAclHtable = null;
+ Table srcAclHtable = null;
+ Table destAclHtable = null;
try {
- srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
- destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
+ srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(TableName.valueOf(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
+ destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
// cube acl
Result result = srcAclHtable.get(new Get(Bytes.toBytes(cubeId)));
@@ -471,7 +472,6 @@ public class CubeMigrationCLI {
destAclHtable.put(put);
}
}
- destAclHtable.flushCommits();
} finally {
IOUtils.closeQuietly(srcAclHtable);
IOUtils.closeQuietly(destAclHtable);
@@ -537,13 +537,12 @@ public class CubeMigrationCLI {
case COPY_ACL: {
String cubeId = (String) opt.params[0];
String modelId = (String) opt.params[1];
- HTableInterface destAclHtable = null;
+ Table destAclHtable = null;
try {
- destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
+ destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
destAclHtable.delete(new Delete(Bytes.toBytes(cubeId)));
destAclHtable.delete(new Delete(Bytes.toBytes(modelId)));
- destAclHtable.flushCommits();
} finally {
IOUtils.closeQuietly(destAclHtable);
}
@@ -560,7 +559,7 @@ public class CubeMigrationCLI {
}
}
- private static void updateMeta(KylinConfig config){
+ private static void updateMeta(KylinConfig config) {
String[] nodes = config.getRestServers();
for (String node : nodes) {
RestClient restClient = new RestClient(node);
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/tool/src/main/java/org/apache/kylin/tool/ExtendCubeToHybridCLI.java
----------------------------------------------------------------------
diff --git a/tool/src/main/java/org/apache/kylin/tool/ExtendCubeToHybridCLI.java b/tool/src/main/java/org/apache/kylin/tool/ExtendCubeToHybridCLI.java
index 19e5db0..f52fc3e 100644
--- a/tool/src/main/java/org/apache/kylin/tool/ExtendCubeToHybridCLI.java
+++ b/tool/src/main/java/org/apache/kylin/tool/ExtendCubeToHybridCLI.java
@@ -25,10 +25,11 @@ import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.persistence.JsonSerializer;
import org.apache.kylin.common.persistence.ResourceStore;
@@ -231,9 +232,9 @@ public class ExtendCubeToHybridCLI {
Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
ProjectInstance project = store.getResource(projectResPath, ProjectInstance.class, projectSerializer);
String projUUID = project.getUuid();
- HTableInterface aclHtable = null;
+ Table aclHtable = null;
try {
- aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(kylinConfig.getMetadataUrlPrefix() + "_acl");
+ aclHtable = HBaseConnection.get(kylinConfig.getStorageUrl()).getTable(TableName.valueOf(kylinConfig.getMetadataUrlPrefix() + "_acl"));
// cube acl
Result result = aclHtable.get(new Get(Bytes.toBytes(origCubeId)));
@@ -253,7 +254,6 @@ public class ExtendCubeToHybridCLI {
aclHtable.put(put);
}
}
- aclHtable.flushCommits();
} finally {
IOUtils.closeQuietly(aclHtable);
}
[4/7] kylin git commit: KYLIN-2375 change default ehcache size to 256M
Posted by li...@apache.org.
KYLIN-2375 change default ehcache size to 256M
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/1e4ae540
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/1e4ae540
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/1e4ae540
Branch: refs/heads/master-hbase1.x
Commit: 1e4ae540bf194c7e67bb622c2eb0cd0ff1a6657c
Parents: 2196975
Author: shaofengshi <sh...@apache.org>
Authored: Thu Jan 19 17:44:19 2017 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Jan 19 17:44:19 2017 +0800
----------------------------------------------------------------------
server/src/main/resources/ehcache-test.xml | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/1e4ae540/server/src/main/resources/ehcache-test.xml
----------------------------------------------------------------------
diff --git a/server/src/main/resources/ehcache-test.xml b/server/src/main/resources/ehcache-test.xml
index eb39774..bffe27a 100644
--- a/server/src/main/resources/ehcache-test.xml
+++ b/server/src/main/resources/ehcache-test.xml
@@ -12,7 +12,7 @@
limitations under the License. See accompanying LICENSE file.
-->
-<ehcache maxBytesLocalHeap="10M">>
+<ehcache maxBytesLocalHeap="256M">>
<cache name="StorageCache"
eternal="false"
timeToIdleSeconds="86400"
[7/7] kylin git commit: KYLIN-1528 Create a branch for v1.5 with
HBase 1.x API
Posted by li...@apache.org.
KYLIN-1528 Create a branch for v1.5 with HBase 1.x API
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/5feb9e92
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/5feb9e92
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/5feb9e92
Branch: refs/heads/master-hbase1.x
Commit: 5feb9e928b561997649f107006052a6a2184d2f7
Parents: 85a1eb3
Author: shaofengshi <sh...@apache.org>
Authored: Wed Mar 23 17:07:05 2016 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Fri Jan 20 18:05:38 2017 +0800
----------------------------------------------------------------------
examples/test_case_data/sandbox/hbase-site.xml | 19 +---
.../kylin/provision/BuildCubeWithEngine.java | 19 ++--
pom.xml | 12 +--
.../kylin/rest/security/AclHBaseStorage.java | 4 +-
.../rest/security/MockAclHBaseStorage.java | 8 +-
.../apache/kylin/rest/security/MockHTable.java | 95 ++++----------------
.../rest/security/RealAclHBaseStorage.java | 9 +-
.../apache/kylin/rest/service/AclService.java | 25 +++---
.../apache/kylin/rest/service/CubeService.java | 35 +++-----
.../apache/kylin/rest/service/QueryService.java | 24 +++--
.../apache/kylin/rest/service/UserService.java | 17 ++--
.../kylin/storage/hbase/HBaseConnection.java | 44 ++++-----
.../kylin/storage/hbase/HBaseResourceStore.java | 31 +++----
.../storage/hbase/cube/SimpleHBaseStore.java | 20 ++---
.../hbase/cube/v2/CubeHBaseEndpointRPC.java | 13 +--
.../storage/hbase/cube/v2/CubeHBaseScanRPC.java | 9 +-
.../coprocessor/endpoint/CubeVisitService.java | 4 +-
.../storage/hbase/steps/CubeHTableUtil.java | 16 ++--
.../storage/hbase/steps/DeprecatedGCStep.java | 24 ++---
.../storage/hbase/steps/HBaseCuboidWriter.java | 7 +-
.../kylin/storage/hbase/steps/MergeGCStep.java | 23 ++---
.../storage/hbase/util/CleanHtableCLI.java | 12 +--
.../storage/hbase/util/CubeMigrationCLI.java | 37 ++++----
.../hbase/util/CubeMigrationCheckCLI.java | 17 ++--
.../hbase/util/DeployCoprocessorCLI.java | 27 +++---
.../hbase/util/ExtendCubeToHybridCLI.java | 8 +-
.../hbase/util/GridTableHBaseBenchmark.java | 34 +++----
.../kylin/storage/hbase/util/HBaseClean.java | 18 ++--
.../hbase/util/HBaseRegionSizeCalculator.java | 35 ++++----
.../kylin/storage/hbase/util/HBaseUsage.java | 9 +-
.../storage/hbase/util/HbaseStreamingInput.java | 30 +++----
.../hbase/util/HtableAlterMetadataCLI.java | 9 +-
.../storage/hbase/util/OrphanHBaseCleanJob.java | 19 ++--
.../kylin/storage/hbase/util/PingHBaseCLI.java | 15 ++--
.../kylin/storage/hbase/util/RowCounterCLI.java | 11 +--
.../storage/hbase/util/StorageCleanupJob.java | 20 +++--
.../storage/hbase/util/UpdateHTableHostCLI.java | 17 ++--
.../org/apache/kylin/tool/CubeMigrationCLI.java | 19 ++--
.../kylin/tool/ExtendCubeToHybridCLI.java | 8 +-
39 files changed, 366 insertions(+), 437 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/examples/test_case_data/sandbox/hbase-site.xml
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/hbase-site.xml b/examples/test_case_data/sandbox/hbase-site.xml
index 46d5345..734908e 100644
--- a/examples/test_case_data/sandbox/hbase-site.xml
+++ b/examples/test_case_data/sandbox/hbase-site.xml
@@ -190,22 +190,5 @@
<name>zookeeper.znode.parent</name>
<value>/hbase-unsecure</value>
</property>
- <property>
- <name>hbase.client.pause</name>
- <value>100</value>
- <description>General client pause value. Used mostly as value to wait
- before running a retry of a failed get, region lookup, etc.
- See hbase.client.retries.number for description of how we backoff from
- this initial pause amount and how this pause works w/ retries.</description>
- </property>
- <property>
- <name>hbase.client.retries.number</name>
- <value>5</value>
- <description>Maximum retries. Used as maximum for all retryable
- operations such as the getting of a cell's value, starting a row update,
- etc. Retry interval is a rough function based on hbase.client.pause. At
- first we retry at this interval but then with backoff, we pretty quickly reach
- retrying every ten seconds. See HConstants#RETRY_BACKOFF for how the backup
- ramps up. Change this setting and hbase.client.pause to suit your workload.</description>
- </property>
+
</configuration>
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
----------------------------------------------------------------------
diff --git a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
index e02bf19..cbec4f6 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
@@ -32,11 +32,9 @@ import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.ClassUtil;
import org.apache.kylin.common.util.HBaseMetadataTestCase;
@@ -58,6 +56,7 @@ import org.apache.kylin.job.impl.threadpool.DefaultScheduler;
import org.apache.kylin.source.ISource;
import org.apache.kylin.source.SourceFactory;
import org.apache.kylin.source.SourcePartition;
+import org.apache.kylin.storage.hbase.HBaseConnection;
import org.apache.kylin.storage.hbase.util.HBaseRegionSizeCalculator;
import org.apache.kylin.storage.hbase.util.ZookeeperJobLock;
import org.apache.kylin.tool.StorageCleanupJob;
@@ -92,10 +91,10 @@ public class BuildCubeWithEngine {
logger.error("error", e);
exitCode = 1;
}
-
+
long millis = System.currentTimeMillis() - start;
System.out.println("Time elapsed: " + (millis / 1000) + " sec - in " + BuildCubeWithEngine.class.getName());
-
+
System.exit(exitCode);
}
@@ -279,7 +278,7 @@ public class BuildCubeWithEngine {
String cubeName = "ci_inner_join_cube";
clearSegment(cubeName);
-
+
SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
f.setTimeZone(TimeZone.getTimeZone("GMT"));
long date1 = 0;
@@ -334,10 +333,10 @@ public class BuildCubeWithEngine {
@SuppressWarnings("unused")
private void checkHFilesInHBase(CubeSegment segment) throws IOException {
- Configuration conf = HBaseConfiguration.create(HadoopUtil.getCurrentConfiguration());
- String tableName = segment.getStorageLocationIdentifier();
- try (HTable table = new HTable(conf, tableName)) {
- HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(table);
+ try (Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl())) {
+ String tableName = segment.getStorageLocationIdentifier();
+
+ HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(tableName, conn);
Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
long totalSize = 0;
for (Long size : sizeMap.values()) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ff4c4e8..34a6e5b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -46,20 +46,20 @@
<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
<!-- Hadoop versions -->
- <hadoop2.version>2.6.0</hadoop2.version>
- <yarn.version>2.6.0</yarn.version>
+ <hadoop2.version>2.7.1</hadoop2.version>
+ <yarn.version>2.7.1</yarn.version>
<!-- Hive versions -->
- <hive.version>0.14.0</hive.version>
- <hive-hcatalog.version>0.14.0</hive-hcatalog.version>
+ <hive.version>1.2.1</hive.version>
+ <hive-hcatalog.version>1.2.1</hive-hcatalog.version>
<!-- HBase versions -->
- <hbase-hadoop2.version>0.98.8-hadoop2</hbase-hadoop2.version>
+ <hbase-hadoop2.version>1.1.1</hbase-hadoop2.version>
<kafka.version>0.10.1.0</kafka.version>
<!-- Hadoop deps, keep compatible with hadoop2.version -->
<zookeeper.version>3.4.6</zookeeper.version>
- <curator.version>2.6.0</curator.version>
+ <curator.version>2.7.1</curator.version>
<jackson.version>2.2.4</jackson.version>
<jsr305.version>3.0.1</jsr305.version>
<guava.version>14.0</guava.version>
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java b/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
index ea68855..8095bf8 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/AclHBaseStorage.java
@@ -20,7 +20,7 @@ package org.apache.kylin.rest.security;
import java.io.IOException;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
/**
*/
@@ -36,6 +36,6 @@ public interface AclHBaseStorage {
String prepareHBaseTable(Class<?> clazz) throws IOException;
- HTableInterface getTable(String tableName) throws IOException;
+ Table getTable(String tableName) throws IOException;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java b/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
index d9326f5..cc76b87 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/MockAclHBaseStorage.java
@@ -21,7 +21,7 @@ package org.apache.kylin.rest.security;
import java.io.IOException;
import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.rest.service.AclService;
import org.apache.kylin.rest.service.QueryService;
@@ -34,8 +34,8 @@ public class MockAclHBaseStorage implements AclHBaseStorage {
private static final String aclTableName = "MOCK-ACL-TABLE";
private static final String userTableName = "MOCK-USER-TABLE";
- private HTableInterface mockedAclTable;
- private HTableInterface mockedUserTable;
+ private Table mockedAclTable;
+ private Table mockedUserTable;
private RealAclHBaseStorage realAcl;
public MockAclHBaseStorage() {
@@ -65,7 +65,7 @@ public class MockAclHBaseStorage implements AclHBaseStorage {
}
@Override
- public HTableInterface getTable(String tableName) throws IOException {
+ public Table getTable(String tableName) throws IOException {
if (realAcl != null) {
return realAcl.getTable(tableName);
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java b/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
index d0aa0ed..972eea9 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/MockHTable.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put;
@@ -91,7 +91,7 @@ import com.google.protobuf.ServiceException;
* <li>remove some methods for loading data, checking values ...</li>
* </ul>
*/
-public class MockHTable implements HTableInterface {
+public class MockHTable implements Table {
private final String tableName;
private final List<String> columnFamilies = new ArrayList<>();
@@ -114,14 +114,6 @@ public class MockHTable implements HTableInterface {
this.columnFamilies.add(columnFamily);
}
- /**
- * {@inheritDoc}
- */
- @Override
- public byte[] getTableName() {
- return tableName.getBytes();
- }
-
@Override
public TableName getName() {
return null;
@@ -200,8 +192,8 @@ public class MockHTable implements HTableInterface {
}
@Override
- public Boolean[] exists(List<Get> gets) throws IOException {
- return new Boolean[0];
+ public boolean[] existsAll(List<Get> list) throws IOException {
+ return new boolean[0];
}
/**
@@ -306,15 +298,6 @@ public class MockHTable implements HTableInterface {
* {@inheritDoc}
*/
@Override
- public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
- // FIXME: implement
- return null;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
public ResultScanner getScanner(Scan scan) throws IOException {
final List<Result> ret = new ArrayList<Result>();
byte[] st = scan.getStartRow();
@@ -446,7 +429,7 @@ public class MockHTable implements HTableInterface {
*/
}
if (filter.hasFilterRow() && !filteredOnRowKey) {
- filter.filterRow(nkvs);
+ filter.filterRow();
}
if (filter.filterRow() || filteredOnRowKey) {
nkvs.clear();
@@ -535,6 +518,11 @@ public class MockHTable implements HTableInterface {
return false;
}
+ @Override
+ public boolean checkAndPut(byte[] bytes, byte[] bytes1, byte[] bytes2, CompareFilter.CompareOp compareOp, byte[] bytes3, Put put) throws IOException {
+ return false;
+ }
+
/**
* {@inheritDoc}
*/
@@ -555,7 +543,7 @@ public class MockHTable implements HTableInterface {
continue;
}
for (KeyValue kv : delete.getFamilyMap().get(family)) {
- if (kv.isDeleteFamily()) {
+ if (kv.isDelete()) {
data.get(row).get(kv.getFamily()).clear();
} else {
data.get(row).get(kv.getFamily()).remove(kv.getQualifier());
@@ -592,6 +580,11 @@ public class MockHTable implements HTableInterface {
return false;
}
+ @Override
+ public boolean checkAndDelete(byte[] bytes, byte[] bytes1, byte[] bytes2, CompareFilter.CompareOp compareOp, byte[] bytes3, Delete delete) throws IOException {
+ return false;
+ }
+
/**
* {@inheritDoc}
*/
@@ -605,7 +598,7 @@ public class MockHTable implements HTableInterface {
*/
@Override
public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount) throws IOException {
- return incrementColumnValue(row, family, qualifier, amount, true);
+ return incrementColumnValue(row, family, qualifier, amount, null);
}
@Override
@@ -617,37 +610,6 @@ public class MockHTable implements HTableInterface {
* {@inheritDoc}
*/
@Override
- public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
- if (check(row, family, qualifier, null)) {
- Put put = new Put(row);
- put.add(family, qualifier, Bytes.toBytes(amount));
- put(put);
- return amount;
- }
- long newValue = Bytes.toLong(data.get(row).get(family).get(qualifier).lastEntry().getValue()) + amount;
- data.get(row).get(family).get(qualifier).put(System.currentTimeMillis(), Bytes.toBytes(newValue));
- return newValue;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public boolean isAutoFlush() {
- return true;
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void flushCommits() throws IOException {
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
public void close() throws IOException {
}
@@ -673,29 +635,6 @@ public class MockHTable implements HTableInterface {
* {@inheritDoc}
*/
@Override
- public void setAutoFlush(boolean autoFlush) {
- throw new NotImplementedException();
-
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
- public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
- throw new NotImplementedException();
-
- }
-
- @Override
- public void setAutoFlushTo(boolean autoFlush) {
- throw new NotImplementedException();
- }
-
- /**
- * {@inheritDoc}
- */
- @Override
public long getWriteBufferSize() {
throw new NotImplementedException();
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java b/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
index 1d520c4..d1a1384 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/security/RealAclHBaseStorage.java
@@ -21,7 +21,8 @@ package org.apache.kylin.rest.security;
import java.io.IOException;
import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.rest.service.AclService;
import org.apache.kylin.rest.service.QueryService;
@@ -58,11 +59,11 @@ public class RealAclHBaseStorage implements AclHBaseStorage {
}
@Override
- public HTableInterface getTable(String tableName) throws IOException {
+ public Table getTable(String tableName) throws IOException {
if (StringUtils.equals(tableName, aclTableName)) {
- return HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+ return HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
} else if (StringUtils.equals(tableName, userTableName)) {
- return HBaseConnection.get(hbaseUrl).getTable(userTableName);
+ return HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
} else {
throw new IllegalStateException("getTable failed" + tableName);
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java b/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
index d693a67..3e3efec 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/AclService.java
@@ -33,7 +33,7 @@ import javax.annotation.PostConstruct;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
@@ -124,7 +124,7 @@ public class AclService implements MutableAclService {
@Override
public List<ObjectIdentity> findChildren(ObjectIdentity parentIdentity) {
List<ObjectIdentity> oids = new ArrayList<ObjectIdentity>();
- HTableInterface htable = null;
+ Table htable = null;
try {
htable = aclHBaseStorage.getTable(aclTableName);
@@ -173,7 +173,7 @@ public class AclService implements MutableAclService {
@Override
public Map<ObjectIdentity, Acl> readAclsById(List<ObjectIdentity> oids, List<Sid> sids) throws NotFoundException {
Map<ObjectIdentity, Acl> aclMaps = new HashMap<ObjectIdentity, Acl>();
- HTableInterface htable = null;
+ Table htable = null;
Result result = null;
try {
htable = aclHBaseStorage.getTable(aclTableName);
@@ -226,17 +226,16 @@ public class AclService implements MutableAclService {
Authentication auth = SecurityContextHolder.getContext().getAuthentication();
PrincipalSid sid = new PrincipalSid(auth);
- HTableInterface htable = null;
+ Table htable = null;
try {
htable = aclHBaseStorage.getTable(aclTableName);
Put put = new Put(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
- put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
- put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
- put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
+ put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
+ put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
+ put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
htable.put(put);
- htable.flushCommits();
logger.debug("ACL of " + objectIdentity + " created successfully.");
} catch (IOException e) {
@@ -250,7 +249,7 @@ public class AclService implements MutableAclService {
@Override
public void deleteAcl(ObjectIdentity objectIdentity, boolean deleteChildren) throws ChildrenExistException {
- HTableInterface htable = null;
+ Table htable = null;
try {
htable = aclHBaseStorage.getTable(aclTableName);
@@ -266,7 +265,6 @@ public class AclService implements MutableAclService {
}
htable.delete(delete);
- htable.flushCommits();
logger.debug("ACL of " + objectIdentity + " deleted successfully.");
} catch (IOException e) {
@@ -284,7 +282,7 @@ public class AclService implements MutableAclService {
throw e;
}
- HTableInterface htable = null;
+ Table htable = null;
try {
htable = aclHBaseStorage.getTable(aclTableName);
@@ -295,17 +293,16 @@ public class AclService implements MutableAclService {
Put put = new Put(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
if (null != acl.getParentAcl()) {
- put.add(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
+ put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
}
for (AccessControlEntry ace : acl.getEntries()) {
AceInfo aceInfo = new AceInfo(ace);
- put.add(Bytes.toBytes(AclHBaseStorage.ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
+ put.addColumn(Bytes.toBytes(AclHBaseStorage.ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
}
if (!put.isEmpty()) {
htable.put(put);
- htable.flushCommits();
logger.debug("ACL of " + acl.getObjectIdentity() + " updated successfully.");
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
index d28c87c..c8c87cb 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -27,9 +27,7 @@ import java.util.List;
import java.util.Map;
import java.util.WeakHashMap;
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.Pair;
import org.apache.kylin.cube.CubeInstance;
@@ -395,33 +393,24 @@ public class CubeService extends BasicService {
if (htableInfoCache.containsKey(tableName)) {
return htableInfoCache.get(tableName);
}
-
- Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
- HTable table = null;
+ Connection conn = HBaseConnection.get(this.getConfig().getStorageUrl());
HBaseResponse hr = null;
long tableSize = 0;
int regionCount = 0;
- try {
- table = new HTable(hconf, tableName);
-
- HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(table);
- Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
+ HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(tableName, conn);
+ Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
- for (long s : sizeMap.values()) {
- tableSize += s;
- }
-
- regionCount = sizeMap.size();
-
- // Set response.
- hr = new HBaseResponse();
- hr.setTableSize(tableSize);
- hr.setRegionCount(regionCount);
- } finally {
- IOUtils.closeQuietly(table);
+ for (long s : sizeMap.values()) {
+ tableSize += s;
}
+ regionCount = sizeMap.size();
+
+ // Set response.
+ hr = new HBaseResponse();
+ hr.setTableSize(tableSize);
+ hr.setRegionCount(regionCount);
htableInfoCache.put(tableName, hr);
return hr;
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 98eb7cb..7ce38ea 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -49,11 +49,11 @@ import javax.sql.DataSource;
import org.apache.calcite.avatica.ColumnMetaData.Rep;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.QueryContext;
import org.apache.kylin.common.debug.BackdoorToggles;
@@ -164,14 +164,13 @@ public class QueryService extends BasicService {
Query[] queryArray = new Query[queries.size()];
byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
- HTableInterface htable = null;
+ Table htable = null;
try {
- htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+ htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
Put put = new Put(Bytes.toBytes(creator));
- put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+ put.addColumn(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
htable.put(put);
- htable.flushCommits();
} finally {
IOUtils.closeQuietly(htable);
}
@@ -197,14 +196,13 @@ public class QueryService extends BasicService {
Query[] queryArray = new Query[queries.size()];
byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
- HTableInterface htable = null;
+ Table htable = null;
try {
- htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+ htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
Put put = new Put(Bytes.toBytes(creator));
- put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+ put.addColumn(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
htable.put(put);
- htable.flushCommits();
} finally {
IOUtils.closeQuietly(htable);
}
@@ -216,12 +214,12 @@ public class QueryService extends BasicService {
}
List<Query> queries = new ArrayList<Query>();
- HTableInterface htable = null;
+ Table htable = null;
try {
- HConnection conn = HBaseConnection.get(hbaseUrl);
+ org.apache.hadoop.hbase.client.Connection conn = HBaseConnection.get(hbaseUrl);
HBaseConnection.createHTableIfNeeded(conn, userTableName, USER_QUERY_FAMILY);
- htable = conn.getTable(userTableName);
+ htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
Get get = new Get(Bytes.toBytes(creator));
get.addFamily(Bytes.toBytes(USER_QUERY_FAMILY));
Result result = htable.get(get);
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java b/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
index 07c7c6f..ab54882 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/UserService.java
@@ -30,11 +30,11 @@ import javax.annotation.PostConstruct;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.kylin.common.util.Bytes;
import org.apache.kylin.common.util.Pair;
import org.apache.kylin.rest.security.AclHBaseStorage;
@@ -72,7 +72,7 @@ public class UserService implements UserDetailsManager {
@Override
public UserDetails loadUserByUsername(String username) throws UsernameNotFoundException {
- HTableInterface htable = null;
+ Table htable = null;
try {
htable = aclHBaseStorage.getTable(userTableName);
@@ -144,16 +144,16 @@ public class UserService implements UserDetailsManager {
@Override
public void updateUser(UserDetails user) {
- HTableInterface htable = null;
+ Table htable = null;
try {
htable = aclHBaseStorage.getTable(userTableName);
Pair<byte[], byte[]> pair = userToHBaseRow(user);
Put put = new Put(pair.getKey());
- put.add(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), pair.getSecond());
+
+ put.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN), pair.getSecond());
htable.put(put);
- htable.flushCommits();
} catch (IOException e) {
throw new RuntimeException(e.getMessage(), e);
} finally {
@@ -163,14 +163,13 @@ public class UserService implements UserDetailsManager {
@Override
public void deleteUser(String username) {
- HTableInterface htable = null;
+ Table htable = null;
try {
htable = aclHBaseStorage.getTable(userTableName);
Delete delete = new Delete(Bytes.toBytes(username));
htable.delete(delete);
- htable.flushCommits();
} catch (IOException e) {
throw new RuntimeException(e.getMessage(), e);
} finally {
@@ -185,7 +184,7 @@ public class UserService implements UserDetailsManager {
@Override
public boolean userExists(String username) {
- HTableInterface htable = null;
+ Table htable = null;
try {
htable = aclHBaseStorage.getTable(userTableName);
@@ -216,7 +215,7 @@ public class UserService implements UserDetailsManager {
s.addColumn(Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_FAMILY), Bytes.toBytes(AclHBaseStorage.USER_AUTHORITY_COLUMN));
List<UserDetails> all = new ArrayList<UserDetails>();
- HTableInterface htable = null;
+ Table htable = null;
ResultScanner scanner = null;
try {
htable = aclHBaseStorage.getTable(userTableName);
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
index 335bfe7..53c95cb 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseConnection.java
@@ -40,9 +40,9 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.kylin.common.KylinConfig;
@@ -64,7 +64,7 @@ public class HBaseConnection {
private static final Logger logger = LoggerFactory.getLogger(HBaseConnection.class);
private static final Map<String, Configuration> configCache = new ConcurrentHashMap<String, Configuration>();
- private static final Map<String, HConnection> connPool = new ConcurrentHashMap<String, HConnection>();
+ private static final Map<String, Connection> connPool = new ConcurrentHashMap<String, Connection>();
private static final ThreadLocal<Configuration> configThreadLocal = new ThreadLocal<>();
private static ExecutorService coprocessorPool = null;
@@ -75,7 +75,7 @@ public class HBaseConnection {
public void run() {
closeCoprocessorPool();
- for (HConnection conn : connPool.values()) {
+ for (Connection conn : connPool.values()) {
try {
conn.close();
} catch (IOException e) {
@@ -144,7 +144,7 @@ public class HBaseConnection {
// using a hbase:xxx URL is deprecated, instead hbase config is always loaded from hbase-site.xml in classpath
if (!(StringUtils.isEmpty(url) || "hbase".equals(url)))
throw new IllegalArgumentException("to use hbase storage, pls set 'kylin.storage.url=hbase' in kylin.properties");
-
+
Configuration conf = HBaseConfiguration.create(HadoopUtil.getCurrentConfiguration());
addHBaseClusterNNHAConfiguration(conf);
@@ -213,9 +213,9 @@ public class HBaseConnection {
// ============================================================================
- // returned HConnection can be shared by multiple threads and does not require close()
+ // returned Connection can be shared by multiple threads and does not require close()
@SuppressWarnings("resource")
- public static HConnection get(String url) {
+ public static Connection get(String url) {
// find configuration
Configuration conf = configCache.get(url);
if (conf == null) {
@@ -223,13 +223,13 @@ public class HBaseConnection {
configCache.put(url, conf);
}
- HConnection connection = connPool.get(url);
+ Connection connection = connPool.get(url);
try {
while (true) {
// I don't use DCL since recreate a connection is not a big issue.
if (connection == null || connection.isClosed()) {
logger.info("connection is null or closed, creating a new one");
- connection = HConnectionManager.createConnection(conf);
+ connection = ConnectionFactory.createConnection(conf);
connPool.put(url, connection);
}
@@ -248,8 +248,8 @@ public class HBaseConnection {
return connection;
}
- public static boolean tableExists(HConnection conn, String tableName) throws IOException {
- HBaseAdmin hbase = new HBaseAdmin(conn);
+ public static boolean tableExists(Connection conn, String tableName) throws IOException {
+ Admin hbase = conn.getAdmin();
try {
return hbase.tableExists(TableName.valueOf(tableName));
} finally {
@@ -269,18 +269,18 @@ public class HBaseConnection {
deleteTable(HBaseConnection.get(hbaseUrl), tableName);
}
- public static void createHTableIfNeeded(HConnection conn, String table, String... families) throws IOException {
- HBaseAdmin hbase = new HBaseAdmin(conn);
-
+ public static void createHTableIfNeeded(Connection conn, String table, String... families) throws IOException {
+ Admin hbase = conn.getAdmin();
+ TableName tableName = TableName.valueOf(table);
try {
if (tableExists(conn, table)) {
logger.debug("HTable '" + table + "' already exists");
- Set<String> existingFamilies = getFamilyNames(hbase.getTableDescriptor(TableName.valueOf(table)));
+ Set<String> existingFamilies = getFamilyNames(hbase.getTableDescriptor(tableName));
boolean wait = false;
for (String family : families) {
if (existingFamilies.contains(family) == false) {
logger.debug("Adding family '" + family + "' to HTable '" + table + "'");
- hbase.addColumn(table, newFamilyDescriptor(family));
+ hbase.addColumn(tableName, newFamilyDescriptor(family));
// addColumn() is async, is there a way to wait it finish?
wait = true;
}
@@ -333,8 +333,8 @@ public class HBaseConnection {
return fd;
}
- public static void deleteTable(HConnection conn, String tableName) throws IOException {
- HBaseAdmin hbase = new HBaseAdmin(conn);
+ public static void deleteTable(Connection conn, String tableName) throws IOException {
+ Admin hbase = conn.getAdmin();
try {
if (!tableExists(conn, tableName)) {
@@ -344,10 +344,10 @@ public class HBaseConnection {
logger.debug("delete HTable '" + tableName + "'");
- if (hbase.isTableEnabled(tableName)) {
- hbase.disableTable(tableName);
+ if (hbase.isTableEnabled(TableName.valueOf(tableName))) {
+ hbase.disableTable(TableName.valueOf(tableName));
}
- hbase.deleteTable(tableName);
+ hbase.deleteTable(TableName.valueOf(tableName));
logger.debug("HTable '" + tableName + "' deleted");
} finally {
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
index 6217350..1c45967 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
@@ -30,14 +30,15 @@ import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.filter.FilterList;
@@ -69,7 +70,7 @@ public class HBaseResourceStore extends ResourceStore {
final String tableNameBase;
final String hbaseUrl;
- HConnection getConnection() throws IOException {
+ Connection getConnection() throws IOException {
return HBaseConnection.get(hbaseUrl);
}
@@ -120,7 +121,7 @@ public class HBaseResourceStore extends ResourceStore {
byte[] endRow = Bytes.toBytes(lookForPrefix);
endRow[endRow.length - 1]++;
- HTableInterface table = getConnection().getTable(getAllInOneTableName());
+ Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
Scan scan = new Scan(startRow, endRow);
if ((filter != null && filter instanceof KeyOnlyFilter) == false) {
scan.addColumn(B_FAMILY, B_COLUMN_TS);
@@ -237,13 +238,12 @@ public class HBaseResourceStore extends ResourceStore {
IOUtils.copy(content, bout);
bout.close();
- HTableInterface table = getConnection().getTable(getAllInOneTableName());
+ Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
try {
byte[] row = Bytes.toBytes(resPath);
Put put = buildPut(resPath, ts, row, bout.toByteArray(), table);
table.put(put);
- table.flushCommits();
} finally {
IOUtils.closeQuietly(table);
}
@@ -251,7 +251,7 @@ public class HBaseResourceStore extends ResourceStore {
@Override
protected long checkAndPutResourceImpl(String resPath, byte[] content, long oldTS, long newTS) throws IOException, IllegalStateException {
- HTableInterface table = getConnection().getTable(getAllInOneTableName());
+ Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
try {
byte[] row = Bytes.toBytes(resPath);
byte[] bOldTS = oldTS == 0 ? null : Bytes.toBytes(oldTS);
@@ -264,8 +264,6 @@ public class HBaseResourceStore extends ResourceStore {
throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + oldTS + ", but it is " + real);
}
- table.flushCommits();
-
return newTS;
} finally {
IOUtils.closeQuietly(table);
@@ -274,7 +272,7 @@ public class HBaseResourceStore extends ResourceStore {
@Override
protected void deleteResourceImpl(String resPath) throws IOException {
- HTableInterface table = getConnection().getTable(getAllInOneTableName());
+ Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
try {
boolean hdfsResourceExist = false;
Result result = internalGetFromHTable(table, resPath, true, false);
@@ -287,7 +285,6 @@ public class HBaseResourceStore extends ResourceStore {
Delete del = new Delete(Bytes.toBytes(resPath));
table.delete(del);
- table.flushCommits();
if (hdfsResourceExist) { // remove hdfs cell value
Path redirectPath = bigCellHDFSPath(resPath);
@@ -308,7 +305,7 @@ public class HBaseResourceStore extends ResourceStore {
}
private Result getFromHTable(String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
- HTableInterface table = getConnection().getTable(getAllInOneTableName());
+ Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
try {
return internalGetFromHTable(table, path, fetchContent, fetchTimestamp);
} finally {
@@ -316,7 +313,7 @@ public class HBaseResourceStore extends ResourceStore {
}
}
- private Result internalGetFromHTable(HTableInterface table, String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
+ private Result internalGetFromHTable(Table table, String path, boolean fetchContent, boolean fetchTimestamp) throws IOException {
byte[] rowkey = Bytes.toBytes(path);
Get get = new Get(rowkey);
@@ -335,7 +332,7 @@ public class HBaseResourceStore extends ResourceStore {
return exists ? result : null;
}
- private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, HTableInterface table) throws IOException {
+ private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, Table table) throws IOException {
Path redirectPath = bigCellHDFSPath(resPath);
FileSystem fileSystem = HadoopUtil.getWorkingFileSystem();
@@ -360,7 +357,7 @@ public class HBaseResourceStore extends ResourceStore {
return redirectPath;
}
- private Put buildPut(String resPath, long ts, byte[] row, byte[] content, HTableInterface table) throws IOException {
+ private Put buildPut(String resPath, long ts, byte[] row, byte[] content, Table table) throws IOException {
int kvSizeLimit = Integer.parseInt(getConnection().getConfiguration().get("hbase.client.keyvalue.maxsize", "10485760"));
if (content.length > kvSizeLimit) {
writeLargeCellToHdfs(resPath, content, table);
@@ -368,8 +365,8 @@ public class HBaseResourceStore extends ResourceStore {
}
Put put = new Put(row);
- put.add(B_FAMILY, B_COLUMN, content);
- put.add(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
+ put.addColumn(B_FAMILY, B_COLUMN, content);
+ put.addColumn(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
return put;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
index b141190..f63d9c2 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/SimpleHBaseStore.java
@@ -26,12 +26,13 @@ import java.util.NoSuchElementException;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.cube.kv.RowConstants;
@@ -86,14 +87,13 @@ public class SimpleHBaseStore implements IGTStore {
}
private class Writer implements IGTWriter {
- final HTableInterface table;
+ final BufferedMutator table;
final ByteBuffer rowkey = ByteBuffer.allocate(50);
final ByteBuffer value = ByteBuffer.allocate(50);
Writer() throws IOException {
- HConnection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
- table = conn.getTable(htableName);
- table.setAutoFlush(false, true);
+ Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+ table = conn.getBufferedMutator(htableName);
}
@Override
@@ -113,24 +113,24 @@ public class SimpleHBaseStore implements IGTStore {
Put put = new Put(rowkey);
put.addImmutable(CF_B, ByteBuffer.wrap(COL_B), HConstants.LATEST_TIMESTAMP, value);
- table.put(put);
+ table.mutate(put);
}
@Override
public void close() throws IOException {
- table.flushCommits();
+ table.flush();
table.close();
}
}
class Reader implements IGTScanner {
- final HTableInterface table;
+ final Table table;
final ResultScanner scanner;
int count = 0;
Reader() throws IOException {
- HConnection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+ Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
table = conn.getTable(htableName);
Scan scan = new Scan();
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
index df1817e..cad5a3f 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseEndpointRPC.java
@@ -26,8 +26,9 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicLong;
import java.util.zip.DataFormatException;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.coprocessor.Batch;
import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
@@ -51,10 +52,10 @@ import org.apache.kylin.storage.gtrecord.StorageResponseGTScatter;
import org.apache.kylin.storage.hbase.HBaseConnection;
import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos;
import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitRequest;
-import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse;
-import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitService;
import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitRequest.IntList;
+import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse;
import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitResponse.Stats;
+import org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.generated.CubeVisitProtos.CubeVisitService;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -117,7 +118,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
final ImmutableBitSet selectedColBlocks = scanRequest.getSelectedColBlocks().set(0);
// globally shared connection, does not require close
- final HConnection conn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
+ final Connection conn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
final List<IntList> hbaseColumnsToGTIntList = Lists.newArrayList();
List<List<Integer>> hbaseColumnsToGT = getHBaseColumnsGTMapping(selectedColBlocks);
@@ -172,7 +173,7 @@ public class CubeHBaseEndpointRPC extends CubeHBaseRPC {
final boolean[] abnormalFinish = new boolean[1];
try {
- HTableInterface table = conn.getTable(cubeSeg.getStorageLocationIdentifier(), HBaseConnection.getCoprocessorPool());
+ Table table = conn.getTable(TableName.valueOf(cubeSeg.getStorageLocationIdentifier()), HBaseConnection.getCoprocessorPool());
final CubeVisitRequest request = builder.build();
final byte[] startKey = epRange.getFirst();
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
index 3cefc5f..a52af90 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/CubeHBaseScanRPC.java
@@ -24,11 +24,12 @@ import java.util.Iterator;
import java.util.List;
import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.kylin.common.util.BytesUtil;
import org.apache.kylin.common.util.ImmutableBitSet;
import org.apache.kylin.common.util.ShardingHash;
@@ -154,8 +155,8 @@ public class CubeHBaseScanRPC extends CubeHBaseRPC {
// primary key (also the 0th column block) is always selected
final ImmutableBitSet selectedColBlocks = scanRequest.getSelectedColBlocks().set(0);
// globally shared connection, does not require close
- HConnection hbaseConn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
- final HTableInterface hbaseTable = hbaseConn.getTable(cubeSeg.getStorageLocationIdentifier());
+ Connection hbaseConn = HBaseConnection.get(cubeSeg.getCubeInstance().getConfig().getStorageUrl());
+ final Table hbaseTable = hbaseConn.getTable(TableName.valueOf(cubeSeg.getStorageLocationIdentifier()));
List<RawScan> rawScans = preparedHBaseScans(scanRequest.getGTScanRanges(), selectedColBlocks);
List<List<Integer>> hbaseColumnsToGT = getHBaseColumnsGTMapping(selectedColBlocks);
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
----------------------------------------------------------------------
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 21a0efb..810747f 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
@@ -142,7 +142,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
if (shardLength == 0) {
return;
}
- byte[] regionStartKey = ArrayUtils.isEmpty(region.getStartKey()) ? new byte[shardLength] : region.getStartKey();
+ byte[] regionStartKey = ArrayUtils.isEmpty(region.getRegionInfo().getStartKey()) ? new byte[shardLength] : region.getRegionInfo().getStartKey();
Bytes.putBytes(rawScan.startKey, 0, regionStartKey, 0, shardLength);
Bytes.putBytes(rawScan.endKey, 0, regionStartKey, 0, shardLength);
}
@@ -179,7 +179,7 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
try (SetThreadName ignored = new SetThreadName("Query %s", queryId)) {
this.serviceStartTime = System.currentTimeMillis();
- region = env.getRegion();
+ region = (HRegion)env.getRegion();
region.startRegionOperation();
// if user change kylin.properties on kylin server, need to manually redeploy coprocessor jar to update KylinConfig of Env.
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
index 2814ad6..feb4842 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
@@ -26,7 +26,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.regionserver.BloomType;
@@ -79,7 +80,8 @@ public class CubeHTableUtil {
tableDesc.setValue(IRealizationConstants.HTableSegmentTag, cubeSegment.toString());
Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
- HBaseAdmin admin = new HBaseAdmin(conf);
+ Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
+ Admin admin = conn.getAdmin();
try {
if (User.isHBaseSecurityEnabled(conf)) {
@@ -92,7 +94,7 @@ public class CubeHTableUtil {
tableDesc.addFamily(cf);
}
- if (admin.tableExists(tableName)) {
+ if (admin.tableExists(TableName.valueOf(tableName))) {
// admin.disableTable(tableName);
// admin.deleteTable(tableName);
throw new RuntimeException("HBase table " + tableName + " exists!");
@@ -101,7 +103,7 @@ public class CubeHTableUtil {
DeployCoprocessorCLI.deployCoprocessor(tableDesc);
admin.createTable(tableDesc, splitKeys);
- Preconditions.checkArgument(admin.isTableAvailable(tableName), "table " + tableName + " created, but is not available due to some reasons");
+ Preconditions.checkArgument(admin.isTableAvailable(TableName.valueOf(tableName)), "table " + tableName + " created, but is not available due to some reasons");
logger.info("create hbase table " + tableName + " done.");
} finally {
IOUtils.closeQuietly(admin);
@@ -110,8 +112,7 @@ public class CubeHTableUtil {
}
public static void deleteHTable(TableName tableName) throws IOException {
- Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
- HBaseAdmin admin = new HBaseAdmin(conf);
+ Admin admin = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getAdmin();
try {
if (admin.tableExists(tableName)) {
logger.info("disabling hbase table " + tableName);
@@ -126,8 +127,7 @@ public class CubeHTableUtil {
/** create a HTable that has the same performance settings as normal cube table, for benchmark purpose */
public static void createBenchmarkHTable(TableName tableName, String cfName) throws IOException {
- Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
- HBaseAdmin admin = new HBaseAdmin(conf);
+ Admin admin = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getAdmin();
try {
if (admin.tableExists(tableName)) {
logger.info("disabling hbase table " + tableName);
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
index eacff9f..df3cf08 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
@@ -25,13 +25,13 @@ import java.util.List;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Bytes;
import org.apache.kylin.common.util.HadoopUtil;
import org.apache.kylin.common.util.HiveCmdBuilder;
import org.apache.kylin.job.exception.ExecuteException;
@@ -100,19 +100,21 @@ public class DeprecatedGCStep extends AbstractExecutable {
List<String> oldTables = getOldHTables();
if (oldTables != null && oldTables.size() > 0) {
String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
- Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
- HBaseAdmin admin = null;
+ Admin admin = null;
try {
- admin = new HBaseAdmin(conf);
+
+ Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+ admin = conn.getAdmin();
+
for (String table : oldTables) {
- if (admin.tableExists(table)) {
- HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
+ if (admin.tableExists(TableName.valueOf(table))) {
+ HTableDescriptor tableDescriptor = admin.getTableDescriptor(TableName.valueOf(table));
String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
if (metadataUrlPrefix.equalsIgnoreCase(host)) {
- if (admin.isTableEnabled(table)) {
- admin.disableTable(table);
+ if (admin.isTableEnabled(TableName.valueOf(table))) {
+ admin.disableTable(TableName.valueOf(table));
}
- admin.deleteTable(table);
+ admin.deleteTable(TableName.valueOf(table));
logger.debug("Dropped HBase table " + table);
output.append("Dropped HBase table " + table + " \n");
} else {
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
index d5b36df..6587d4e 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseCuboidWriter.java
@@ -23,8 +23,8 @@ import java.util.List;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.kylin.common.util.ImmutableBitSet;
import org.apache.kylin.cube.CubeSegment;
import org.apache.kylin.cube.cuboid.Cuboid;
@@ -49,7 +49,7 @@ public class HBaseCuboidWriter implements ICuboidWriter {
private final List<KeyValueCreator> keyValueCreators;
private final int nColumns;
- private final HTableInterface hTable;
+ private final Table hTable;
private final CubeDesc cubeDesc;
private final CubeSegment cubeSegment;
private final Object[] measureValues;
@@ -58,7 +58,7 @@ public class HBaseCuboidWriter implements ICuboidWriter {
private AbstractRowKeyEncoder rowKeyEncoder;
private byte[] keybuf;
- public HBaseCuboidWriter(CubeSegment segment, HTableInterface hTable) {
+ public HBaseCuboidWriter(CubeSegment segment, Table hTable) {
this.keyValueCreators = Lists.newArrayList();
this.cubeSegment = segment;
this.cubeDesc = cubeSegment.getCubeDesc();
@@ -117,7 +117,6 @@ public class HBaseCuboidWriter implements ICuboidWriter {
long t = System.currentTimeMillis();
if (hTable != null) {
hTable.put(puts);
- hTable.flushCommits();
}
logger.info("commit total " + puts.size() + " puts, totally cost:" + (System.currentTimeMillis() - t) + "ms");
puts.clear();
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
index 5b2441c..2f7e164 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
@@ -24,11 +24,11 @@ import java.util.Collections;
import java.util.List;
import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Bytes;
import org.apache.kylin.job.exception.ExecuteException;
import org.apache.kylin.job.execution.AbstractExecutable;
import org.apache.kylin.job.execution.ExecutableContext;
@@ -69,19 +69,20 @@ public class MergeGCStep extends AbstractExecutable {
List<String> oldTables = getOldHTables();
if (oldTables != null && oldTables.size() > 0) {
String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
- Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
- HBaseAdmin admin = null;
+ Admin admin = null;
try {
- admin = new HBaseAdmin(conf);
+ Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+ admin = conn.getAdmin();
+
for (String table : oldTables) {
- if (admin.tableExists(table)) {
- HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
+ if (admin.tableExists(TableName.valueOf(table))) {
+ HTableDescriptor tableDescriptor = admin.getTableDescriptor(TableName.valueOf((table)));
String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
if (metadataUrlPrefix.equalsIgnoreCase(host)) {
- if (admin.isTableEnabled(table)) {
- admin.disableTable(table);
+ if (admin.isTableEnabled(TableName.valueOf(table))) {
+ admin.disableTable(TableName.valueOf(table));
}
- admin.deleteTable(table);
+ admin.deleteTable(TableName.valueOf(table));
logger.debug("Dropped htable: " + table);
output.append("HBase table " + table + " is dropped. \n");
} else {
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
index a150607..56f867a 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
@@ -21,9 +21,11 @@ package org.apache.kylin.storage.hbase.util;
import java.io.IOException;
import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.AbstractApplication;
import org.apache.kylin.common.util.OptionsHelper;
import org.apache.kylin.metadata.realization.IRealizationConstants;
@@ -38,8 +40,8 @@ public class CleanHtableCLI extends AbstractApplication {
protected static final Logger logger = LoggerFactory.getLogger(CleanHtableCLI.class);
private void clean() throws IOException {
- Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
- HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+ Connection conn = HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+ Admin hbaseAdmin = conn.getAdmin();
for (HTableDescriptor descriptor : hbaseAdmin.listTables()) {
String name = descriptor.getNameAsString().toLowerCase();
@@ -50,7 +52,7 @@ public class CleanHtableCLI extends AbstractApplication {
System.out.println();
descriptor.setValue(IRealizationConstants.HTableOwner, "DL-eBay-Kylin@ebay.com");
- hbaseAdmin.modifyTable(descriptor.getNameAsString(), descriptor);
+ hbaseAdmin.modifyTable(TableName.valueOf(descriptor.getNameAsString()), descriptor);
}
}
hbaseAdmin.close();
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
index 68c0a39..581de38 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
@@ -26,19 +26,19 @@ import java.util.Map;
import java.util.Set;
import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.persistence.JsonSerializer;
import org.apache.kylin.common.persistence.RawResource;
@@ -89,7 +89,7 @@ public class CubeMigrationCLI {
private static ResourceStore srcStore;
private static ResourceStore dstStore;
private static FileSystem hdfsFS;
- private static HBaseAdmin hbaseAdmin;
+ private static Admin hbaseAdmin;
public static final String ACL_INFO_FAMILY = "i";
private static final String ACL_TABLE_NAME = "_acl";
@@ -134,8 +134,8 @@ public class CubeMigrationCLI {
checkAndGetHbaseUrl();
- Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
- hbaseAdmin = new HBaseAdmin(conf);
+ Connection conn = HBaseConnection.get(srcConfig.getStorageUrl());
+ hbaseAdmin = conn.getAdmin();
hdfsFS = HadoopUtil.getWorkingFileSystem();
@@ -233,6 +233,7 @@ public class CubeMigrationCLI {
operations.add(new Opt(OptType.COPY_DICT_OR_SNAPSHOT, new Object[] { item, cube.getName() }));
}
}
+
private static void addCubeAndModelIntoProject(CubeInstance srcCube, String cubeName, String projectName) throws IOException {
String projectResPath = ProjectInstance.concatResourcePath(projectName);
if (!dstStore.exists(projectResPath))
@@ -326,8 +327,8 @@ public class CubeMigrationCLI {
switch (opt.type) {
case CHANGE_HTABLE_HOST: {
- String tableName = (String) opt.params[0];
- HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+ TableName tableName = TableName.valueOf((String) opt.params[0]);
+ HTableDescriptor desc = hbaseAdmin.getTableDescriptor(tableName);
hbaseAdmin.disableTable(tableName);
desc.setValue(IRealizationConstants.HTableTag, dstConfig.getMetadataUrlPrefix());
hbaseAdmin.modifyTable(tableName, desc);
@@ -449,11 +450,11 @@ public class CubeMigrationCLI {
Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
String projUUID = project.getUuid();
- HTableInterface srcAclHtable = null;
- HTableInterface destAclHtable = null;
+ Table srcAclHtable = null;
+ Table destAclHtable = null;
try {
- srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
- destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
+ srcAclHtable = HBaseConnection.get(srcConfig.getStorageUrl()).getTable(TableName.valueOf(srcConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
+ destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
// cube acl
Result result = srcAclHtable.get(new Get(Bytes.toBytes(cubeId)));
@@ -473,7 +474,6 @@ public class CubeMigrationCLI {
destAclHtable.put(put);
}
}
- destAclHtable.flushCommits();
} finally {
IOUtils.closeQuietly(srcAclHtable);
IOUtils.closeQuietly(destAclHtable);
@@ -504,8 +504,8 @@ public class CubeMigrationCLI {
switch (opt.type) {
case CHANGE_HTABLE_HOST: {
- String tableName = (String) opt.params[0];
- HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+ TableName tableName = TableName.valueOf((String) opt.params[0]);
+ HTableDescriptor desc = hbaseAdmin.getTableDescriptor(tableName);
hbaseAdmin.disableTable(tableName);
desc.setValue(IRealizationConstants.HTableTag, srcConfig.getMetadataUrlPrefix());
hbaseAdmin.modifyTable(tableName, desc);
@@ -539,13 +539,12 @@ public class CubeMigrationCLI {
case COPY_ACL: {
String cubeId = (String) opt.params[0];
String modelId = (String) opt.params[1];
- HTableInterface destAclHtable = null;
+ Table destAclHtable = null;
try {
- destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME);
+ destAclHtable = HBaseConnection.get(dstConfig.getStorageUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + ACL_TABLE_NAME));
destAclHtable.delete(new Delete(Bytes.toBytes(cubeId)));
destAclHtable.delete(new Delete(Bytes.toBytes(modelId)));
- destAclHtable.flushCommits();
} finally {
IOUtils.closeQuietly(destAclHtable);
}
@@ -562,7 +561,7 @@ public class CubeMigrationCLI {
}
}
- private static void updateMeta(KylinConfig config){
+ private static void updateMeta(KylinConfig config) {
String[] nodes = config.getRestServers();
for (String node : nodes) {
RestClient restClient = new RestClient(node);
http://git-wip-us.apache.org/repos/asf/kylin/blob/5feb9e92/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
index 8bd4abf..20d0f7d 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCheckCLI.java
@@ -26,10 +26,10 @@ import org.apache.commons.cli.Option;
import org.apache.commons.cli.OptionBuilder;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.OptionsHelper;
import org.apache.kylin.cube.CubeInstance;
@@ -61,7 +61,7 @@ public class CubeMigrationCheckCLI {
private static final Option OPTION_CUBE = OptionBuilder.withArgName("cube").hasArg().isRequired(false).withDescription("The name of cube migrated").create("cube");
private KylinConfig dstCfg;
- private HBaseAdmin hbaseAdmin;
+ private Admin hbaseAdmin;
private List<String> issueExistHTables;
private List<String> inconsistentHTables;
@@ -130,9 +130,8 @@ public class CubeMigrationCheckCLI {
this.dstCfg = kylinConfig;
this.ifFix = isFix;
- Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
- hbaseAdmin = new HBaseAdmin(conf);
-
+ Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
+ hbaseAdmin = conn.getAdmin();
issueExistHTables = Lists.newArrayList();
inconsistentHTables = Lists.newArrayList();
}
@@ -189,10 +188,10 @@ public class CubeMigrationCheckCLI {
String[] sepNameList = segFullName.split(",");
HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(sepNameList[0]));
logger.info("Change the host of htable " + sepNameList[0] + "belonging to cube " + sepNameList[1] + " from " + desc.getValue(IRealizationConstants.HTableTag) + " to " + dstCfg.getMetadataUrlPrefix());
- hbaseAdmin.disableTable(sepNameList[0]);
+ hbaseAdmin.disableTable(TableName.valueOf(sepNameList[0]));
desc.setValue(IRealizationConstants.HTableTag, dstCfg.getMetadataUrlPrefix());
- hbaseAdmin.modifyTable(sepNameList[0], desc);
- hbaseAdmin.enableTable(sepNameList[0]);
+ hbaseAdmin.modifyTable(TableName.valueOf(sepNameList[0]), desc);
+ hbaseAdmin.enableTable(TableName.valueOf(sepNameList[0]));
}
} else {
logger.info("------ Inconsistent HTables Needed To Be Fixed ------");
[2/7] kylin git commit: KYLIN-2409 change inmem cubing to single
thread by default
Posted by li...@apache.org.
KYLIN-2409 change inmem cubing to single thread by default
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/21969753
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/21969753
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/21969753
Branch: refs/heads/master-hbase1.x
Commit: 21969753cc66efa22bc9fc933af46346e9846631
Parents: b49c9e3
Author: shaofengshi <sh...@apache.org>
Authored: Thu Jan 19 11:52:17 2017 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Jan 19 17:37:42 2017 +0800
----------------------------------------------------------------------
.../main/java/org/apache/kylin/common/KylinConfigBase.java | 6 +++++-
.../kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java | 2 +-
.../java/org/apache/kylin/gridtable/GTAggregateScanner.java | 9 +++++++++
.../org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java | 2 ++
4 files changed, 17 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/21969753/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index d6774ff..74903d5 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -300,6 +300,10 @@ abstract public class KylinConfigBase implements Serializable {
return Integer.parseInt(getOptional("kylin.cube.algorithm.inmem-split-limit", "500"));
}
+ public int getCubeAlgorithmInMemConcurrentThreads() {
+ return Integer.parseInt(getOptional("kylin.cube.algorithm.inmem-concurrent-threads", "1"));
+ }
+
public boolean isIgnoreCubeSignatureInconsistency() {
return Boolean.parseBoolean(getOptional("kylin.cube.ignore-signature-inconsistency", "false"));
}
@@ -744,7 +748,7 @@ abstract public class KylinConfigBase implements Serializable {
}
public int getYarnStatusCheckIntervalSeconds() {
- return Integer.parseInt(getOptional("kylin.engine.mr.yarn-check-interval-seconds", "60"));
+ return Integer.parseInt(getOptional("kylin.engine.mr.yarn-check-interval-seconds", "10"));
}
// ============================================================================
http://git-wip-us.apache.org/repos/asf/kylin/blob/21969753/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
index 651203a..c7a4a05 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/AbstractInMemCubeBuilder.java
@@ -46,7 +46,7 @@ abstract public class AbstractInMemCubeBuilder {
final protected CubeDesc cubeDesc;
final protected Map<TblColRef, Dictionary<String>> dictionaryMap;
- protected int taskThreadCount = 4;
+ protected int taskThreadCount = 1;
protected int reserveMemoryMB = 100;
public AbstractInMemCubeBuilder(CubeDesc cubeDesc, IJoinedFlatTableDesc flatDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/21969753/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
----------------------------------------------------------------------
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 55c04c6..9158aa3 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
@@ -36,6 +36,7 @@ import java.util.Map.Entry;
import org.apache.commons.io.IOUtils;
import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Bytes;
import org.apache.kylin.common.util.ImmutableBitSet;
import org.apache.kylin.common.util.MemoryBudgetController;
import org.apache.kylin.common.util.Pair;
@@ -178,11 +179,16 @@ public class GTAggregateScanner implements IGTScanner {
final List<Dump> dumps;
final int keyLength;
final boolean[] compareMask;
+ boolean compareAll = true;
final BufferedMeasureCodec measureCodec;
final Comparator<byte[]> bytesComparator = new Comparator<byte[]>() {
@Override
public int compare(byte[] o1, byte[] o2) {
+ if (compareAll) {
+ return Bytes.compareTo(o1, o2);
+ }
+
int result = 0;
// profiler shows this check is slow
// Preconditions.checkArgument(keyLength == o1.length && keyLength == o2.length);
@@ -206,6 +212,9 @@ public class GTAggregateScanner implements IGTScanner {
public AggregationCache() {
compareMask = createCompareMask();
+ for (boolean l : compareMask) {
+ compareAll = compareAll && l;
+ }
keyLength = compareMask.length;
dumps = Lists.newArrayList();
aggBufMap = createBuffMap();
http://git-wip-us.apache.org/repos/asf/kylin/blob/21969753/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
index 116d5e0..c0ff2f2 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/InMemCuboidMapper.java
@@ -94,8 +94,10 @@ public class InMemCuboidMapper<KEYIN> extends KylinMapper<KEYIN, Object, ByteArr
dictionaryMap.put(col, cubeSegment.getDictionary(col));
}
+ int taskCount = config.getCubeAlgorithmInMemConcurrentThreads();
DoggedCubeBuilder cubeBuilder = new DoggedCubeBuilder(cube.getDescriptor(), flatDesc, dictionaryMap);
cubeBuilder.setReserveMemoryMB(calculateReserveMB(context.getConfiguration()));
+ cubeBuilder.setConcurrentThreads(taskCount);
ExecutorService executorService = Executors.newSingleThreadExecutor();
future = executorService.submit(cubeBuilder.buildAsRunnable(queue, new MapContextGTRecordWriter(context, cubeDesc, cubeSegment)));
[5/7] kylin git commit: minor,
fix false positive warning in test output
Posted by li...@apache.org.
minor, fix false positive warning in test output
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/85a1eb39
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/85a1eb39
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/85a1eb39
Branch: refs/heads/master-hbase1.x
Commit: 85a1eb399d8dcd099d4607aa802faa5ccdf4d2bf
Parents: 1e4ae54
Author: Billy Liu <bi...@apache.org>
Authored: Fri Jan 20 08:13:11 2017 +0800
Committer: Billy Liu <bi...@apache.org>
Committed: Fri Jan 20 08:13:11 2017 +0800
----------------------------------------------------------------------
.../org/apache/kylin/common/util/AbstractKylinTestCase.java | 7 ++++++-
.../java/org/apache/kylin/dict/lookup/SnapshotManager.java | 6 ++++++
.../src/main/java/org/apache/kylin/job/dao/ExecutableDao.java | 6 ++++++
.../java/org/apache/kylin/metadata/cachesync/Broadcaster.java | 2 +-
examples/test_case_data/localmeta/cube_desc/ssb.json | 4 ++--
examples/test_case_data/localmeta/cube_desc/ssb_cube1.json | 4 ++--
examples/test_case_data/localmeta/cube_desc/ssb_cube2.json | 4 ++--
examples/test_case_data/localmeta/cube_desc/ssb_cube3.json | 4 ++--
.../cube_desc/test_kylin_cube_without_slr_left_join_desc.json | 4 ++--
.../localmeta/cube_desc/test_streaming_table_cube_desc.json | 2 +-
examples/test_case_data/localmeta/kylin.properties | 2 +-
examples/test_case_data/sandbox/kylin.properties | 2 +-
kylin-it/pom.xml | 2 +-
13 files changed, 33 insertions(+), 16 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/85a1eb39/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
----------------------------------------------------------------------
diff --git a/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java b/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
index 2154c32..4a9804d 100644
--- a/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
+++ b/core-common/src/test/java/org/apache/kylin/common/util/AbstractKylinTestCase.java
@@ -31,12 +31,17 @@ public abstract class AbstractKylinTestCase {
public static final String[] SERVICES_WITH_CACHE = { //
"org.apache.kylin.cube.CubeManager", //
"org.apache.kylin.cube.CubeDescManager", //
+ "org.apache.kylin.dict.lookup.SnapshotManager", //
+ "org.apache.kylin.dict.DictionaryManager", //
"org.apache.kylin.storage.hybrid.HybridManager", //
"org.apache.kylin.metadata.realization.RealizationRegistry", //
"org.apache.kylin.metadata.project.ProjectManager", //
"org.apache.kylin.metadata.MetadataManager", //
+ "org.apache.kylin.metadata.cachesync.Broadcaster", //
+ "org.apache.kylin.metadata.badquery.BadQueryHistoryManager", //
"org.apache.kylin.job.impl.threadpool.DistributedScheduler", //
- "org.apache.kylin.job.manager.ExecutableManager", //
+ "org.apache.kylin.job.execution.ExecutableManager", //
+ "org.apache.kylin.job.dao.ExecutableDao" //
};
public abstract void createTestMetadata() throws Exception;
http://git-wip-us.apache.org/repos/asf/kylin/blob/85a1eb39/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotManager.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotManager.java b/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotManager.java
index 085158a..b45d017 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotManager.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/lookup/SnapshotManager.java
@@ -66,6 +66,12 @@ public class SnapshotManager {
return r;
}
+ public static void clearCache() {
+ synchronized (SERVICE_CACHE) {
+ SERVICE_CACHE.clear();
+ }
+ }
+
// ============================================================================
private KylinConfig config;
http://git-wip-us.apache.org/repos/asf/kylin/blob/85a1eb39/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java b/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
index 5cae5ac..96505e6 100644
--- a/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
+++ b/core-job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
@@ -64,6 +64,12 @@ public class ExecutableDao {
return r;
}
+ public static void clearCache() {
+ synchronized (CACHE) {
+ CACHE.clear();
+ }
+ }
+
private ExecutableDao(KylinConfig config) {
logger.info("Using metadata url: " + config);
this.store = MetadataManager.getInstance(config).getStore();
http://git-wip-us.apache.org/repos/asf/kylin/blob/85a1eb39/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
index d00c490..17b644d 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
@@ -83,7 +83,7 @@ public class Broadcaster {
}
// call Broadcaster.getInstance().notifyClearAll() to clear cache
- static void clearCache() {
+ public static void clearCache() {
synchronized (CACHE) {
CACHE.clear();
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/85a1eb39/examples/test_case_data/localmeta/cube_desc/ssb.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/ssb.json b/examples/test_case_data/localmeta/cube_desc/ssb.json
index a13ac53..280fc82 100644
--- a/examples/test_case_data/localmeta/cube_desc/ssb.json
+++ b/examples/test_case_data/localmeta/cube_desc/ssb.json
@@ -183,7 +183,7 @@
"engine_type" : 2,
"storage_type" : 2,
"override_kylin_properties" : {
- "kylin.hbase.default.compression.codec" : "lz4",
- "kylin.cube.aggrgroup.isMandatoryOnlyValid" : "true"
+ "kylin.storage.hbase.compression-codec" : "lz4",
+ "kylin.cube.aggrgroup.is-mandatory-only-valid" : "true"
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kylin/blob/85a1eb39/examples/test_case_data/localmeta/cube_desc/ssb_cube1.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/ssb_cube1.json b/examples/test_case_data/localmeta/cube_desc/ssb_cube1.json
index 5986057..861e67d 100644
--- a/examples/test_case_data/localmeta/cube_desc/ssb_cube1.json
+++ b/examples/test_case_data/localmeta/cube_desc/ssb_cube1.json
@@ -149,7 +149,7 @@
"engine_type" : 2,
"storage_type" : 2,
"override_kylin_properties" : {
- "kylin.hbase.default.compression.codec" : "lz4",
- "kylin.cube.aggrgroup.isMandatoryOnlyValid" : "true"
+ "kylin.storage.hbase.compression-codec" : "lz4",
+ "kylin.cube.aggrgroup.is-mandatory-only-valid" : "true"
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kylin/blob/85a1eb39/examples/test_case_data/localmeta/cube_desc/ssb_cube2.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/ssb_cube2.json b/examples/test_case_data/localmeta/cube_desc/ssb_cube2.json
index fc75ea8..c2bf3c0 100644
--- a/examples/test_case_data/localmeta/cube_desc/ssb_cube2.json
+++ b/examples/test_case_data/localmeta/cube_desc/ssb_cube2.json
@@ -149,7 +149,7 @@
"engine_type" : 2,
"storage_type" : 2,
"override_kylin_properties" : {
- "kylin.hbase.default.compression.codec" : "lz4",
- "kylin.cube.aggrgroup.isMandatoryOnlyValid" : "true"
+ "kylin.storage.hbase.compression-codec" : "lz4",
+ "kylin.cube.aggrgroup.is-mandatory-only-valid" : "true"
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kylin/blob/85a1eb39/examples/test_case_data/localmeta/cube_desc/ssb_cube3.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/ssb_cube3.json b/examples/test_case_data/localmeta/cube_desc/ssb_cube3.json
index 4509116..d45b6a7 100644
--- a/examples/test_case_data/localmeta/cube_desc/ssb_cube3.json
+++ b/examples/test_case_data/localmeta/cube_desc/ssb_cube3.json
@@ -137,7 +137,7 @@
"engine_type" : 2,
"storage_type" : 2,
"override_kylin_properties" : {
- "kylin.hbase.default.compression.codec" : "lz4",
- "kylin.cube.aggrgroup.isMandatoryOnlyValid" : "true"
+ "kylin.storage.hbase.compression-codec" : "lz4",
+ "kylin.cube.aggrgroup.is-mandatory-only-valid" : "true"
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kylin/blob/85a1eb39/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
index 9aeaa2e..25b66f2 100644
--- a/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
+++ b/examples/test_case_data/localmeta/cube_desc/test_kylin_cube_without_slr_left_join_desc.json
@@ -287,8 +287,8 @@
"engine_type" : 2,
"storage_type" : 2,
"override_kylin_properties": {
- "kylin.job.cubing.inmem.sampling.hll.precision": "16",
- "kylin.hive.create.flat.table.method": "2"
+ "kylin.job.sampling-hll-precision": "16",
+ "kylin.source.hive.create-flat-table-method": "2"
},
"partition_date_start": 0
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/85a1eb39/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json b/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json
index 640b91c..0065b02 100644
--- a/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json
+++ b/examples/test_case_data/localmeta/cube_desc/test_streaming_table_cube_desc.json
@@ -106,7 +106,7 @@
} ],
"override_kylin_properties": {
"kylin.cube.algorithm": "inmem",
- "kylin.cube.building.segment.max": "5"
+ "kylin.cube.max-building-segments": "5"
},
"notify_list" : [ ],
"status_need_notify" : [ ],
http://git-wip-us.apache.org/repos/asf/kylin/blob/85a1eb39/examples/test_case_data/localmeta/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/localmeta/kylin.properties b/examples/test_case_data/localmeta/kylin.properties
index f4c6772..e89c767 100644
--- a/examples/test_case_data/localmeta/kylin.properties
+++ b/examples/test_case_data/localmeta/kylin.properties
@@ -133,4 +133,4 @@ kylin.engine.mr.config-override.test2=test2
kylin.job.lock=org.apache.kylin.job.lock.MockJobLock
kylin.engine.provider.0=org.apache.kylin.engine.mr.MRBatchCubingEngine
-kylin.cube.engine.2=org.apache.kylin.engine.mr.MRBatchCubingEngine2
\ No newline at end of file
+kylin.engine.provider.2=org.apache.kylin.engine.mr.MRBatchCubingEngine2
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kylin/blob/85a1eb39/examples/test_case_data/sandbox/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin.properties b/examples/test_case_data/sandbox/kylin.properties
index 06f8e4b..6c512dc 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -25,7 +25,7 @@ kylin.server.mode=all
kylin.storage.hbase.owner-tag=whoami@kylin.apache.org
# List of web servers in use, this enables one web server instance to sync up with other servers.
-kylin.server.cluster-servers=localhost:7070
+#kylin.server.cluster-servers=localhost:7070
# Display timezone on UI,format like[GMT+N or GMT-N]
kylin.web.timezone=GMT+8
http://git-wip-us.apache.org/repos/asf/kylin/blob/85a1eb39/kylin-it/pom.xml
----------------------------------------------------------------------
diff --git a/kylin-it/pom.xml b/kylin-it/pom.xml
index 080558b..f972b4b 100644
--- a/kylin-it/pom.xml
+++ b/kylin-it/pom.xml
@@ -282,7 +282,7 @@
<value>file:${project.basedir}/..//build/conf/kylin-tools-log4j.properties</value>
</property>
</systemProperties>
- <argLine>-Xms1G -Xmx2G -XX:PermSize=128M -XX:MaxPermSize=512M</argLine>
+ <argLine>-Xms1G -Xmx2G -XX:PermSize=128M -XX:MaxPermSize=512M -Dkylin.server.cluster-servers=localhost:7070</argLine>
</configuration>
</plugin>
<plugin>
[3/7] kylin git commit: KYLIN-2411 Kill MR job on pause
Posted by li...@apache.org.
KYLIN-2411 Kill MR job on pause
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/b49c9e39
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/b49c9e39
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/b49c9e39
Branch: refs/heads/master-hbase1.x
Commit: b49c9e3951c29a21e71f179ee77e2295fbc52ba9
Parents: 38c3e7b
Author: shaofengshi <sh...@apache.org>
Authored: Thu Jan 19 11:02:12 2017 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Jan 19 17:37:42 2017 +0800
----------------------------------------------------------------------
.../org/apache/kylin/job/execution/ExecutableManager.java | 2 +-
.../apache/kylin/engine/mr/common/MapReduceExecutable.java | 8 ++++++--
2 files changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/b49c9e39/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java b/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java
index 466cdad..48cedb5 100644
--- a/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java
@@ -347,7 +347,7 @@ public class ExecutableManager {
for (AbstractExecutable task : tasks) {
if (task.getId().compareTo(stepId) >= 0) {
logger.debug("rollback task : " + task);
- updateJobOutput(task.getId(), ExecutableState.READY, null, null);
+ updateJobOutput(task.getId(), ExecutableState.READY, Maps.<String, String>newHashMap(), "");
}
}
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/b49c9e39/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceExecutable.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceExecutable.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceExecutable.java
index f887c4c..6de07ca 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceExecutable.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/MapReduceExecutable.java
@@ -148,7 +148,7 @@ public class MapReduceExecutable extends AbstractExecutable {
// boolean useKerberosAuth = context.getConfig().isGetJobStatusWithKerberos();
// HadoopStatusChecker statusChecker = new HadoopStatusChecker(restStatusCheckUrl, mrJobId, output, useKerberosAuth);
JobStepStatusEnum status = JobStepStatusEnum.NEW;
- while (!isDiscarded()) {
+ while (!isDiscarded() && !isPaused()) {
JobStepStatusEnum newStatus = HadoopJobStatusChecker.checkStatus(job, output);
if (status == JobStepStatusEnum.KILLED) {
@@ -184,7 +184,11 @@ public class MapReduceExecutable extends AbstractExecutable {
}
}
- return new ExecuteResult(ExecuteResult.State.DISCARDED, output.toString());
+ if (isDiscarded()) {
+ return new ExecuteResult(ExecuteResult.State.DISCARDED, output.toString());
+ } else {
+ return new ExecuteResult(ExecuteResult.State.STOPPED, output.toString());
+ }
} catch (ReflectiveOperationException e) {
logger.error("error getMapReduceJobClass, class name:" + getParam(KEY_MR_JOB), e);