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/02/16 00:53:00 UTC
[12/17] kylin git commit: minor,
add new interface in IMRInput to support partial table data
minor, add new interface in IMRInput to support partial table data
Signed-off-by: shaofengshi <sh...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/545d1455
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/545d1455
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/545d1455
Branch: refs/heads/master-hbase0.98
Commit: 545d14552da0e069fcbffced6c51fa19db0cba3f
Parents: f474ac5
Author: Cheng Wang <ch...@kyligence.io>
Authored: Tue Feb 14 14:51:33 2017 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Wed Feb 15 13:14:23 2017 +0800
----------------------------------------------------------------------
.../java/org/apache/kylin/metadata/model/TableDesc.java | 10 ----------
.../main/java/org/apache/kylin/engine/mr/IMRInput.java | 2 ++
.../src/main/java/org/apache/kylin/engine/mr/MRUtil.java | 8 ++++++++
.../java/org/apache/kylin/source/hive/HiveMRInput.java | 11 ++++++++---
.../java/org/apache/kylin/source/kafka/KafkaMRInput.java | 7 ++++++-
5 files changed, 24 insertions(+), 14 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/545d1455/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java
index be62062..b4d49fb 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableDesc.java
@@ -46,8 +46,6 @@ public class TableDesc extends RootPersistentEntity implements ISourceAware {
private ColumnDesc[] columns;
@JsonProperty("source_type")
private int sourceType = ISourceAware.ID_HIVE;
- @JsonProperty("whole_scan")
- private boolean wholeScan = true;
@JsonProperty("table_type")
private String tableType;
@@ -104,14 +102,6 @@ public class TableDesc extends RootPersistentEntity implements ISourceAware {
return identity;
}
- public void setWholeScan(boolean wholeScan) {
- this.wholeScan = wholeScan;
- }
-
- public boolean getWholeScan() {
- return this.wholeScan;
- }
-
public boolean isView() {
return TABLE_TYPE_VIRTUAL_VIEW.equals(tableType);
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/545d1455/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
index 62cede9..f6578f9 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/IMRInput.java
@@ -35,6 +35,8 @@ public interface IMRInput {
/** Return an InputFormat that reads from specified table. */
public IMRTableInputFormat getTableInputFormat(TableDesc table);
+ public IMRTableInputFormat getTableInputFormat(TableDesc table, boolean isPartial);
+
/** Return a helper to participate in batch cubing merge job flow. */
public IMRBatchMergeInputSide getBatchMergeInputSide(ISegment seg);
http://git-wip-us.apache.org/repos/asf/kylin/blob/545d1455/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
index 67eef5e..eb7bc4b 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/MRUtil.java
@@ -47,10 +47,18 @@ public class MRUtil {
return getTableInputFormat(getTableDesc(tableName));
}
+ public static IMRTableInputFormat getTableInputFormat(String tableName, boolean isPartial) {
+ return getTableInputFormat(getTableDesc(tableName), isPartial);
+ }
+
public static IMRTableInputFormat getTableInputFormat(TableDesc tableDesc) {
return SourceFactory.createEngineAdapter(tableDesc, IMRInput.class).getTableInputFormat(tableDesc);
}
+ public static IMRTableInputFormat getTableInputFormat(TableDesc tableDesc, boolean isPartial) {
+ return SourceFactory.createEngineAdapter(tableDesc, IMRInput.class).getTableInputFormat(tableDesc, isPartial);
+ }
+
private static TableDesc getTableDesc(String tableName) {
return MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(tableName);
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/545d1455/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
index f9c7cfc..c1ea45a 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
@@ -60,8 +60,8 @@ import com.google.common.collect.Sets;
public class HiveMRInput implements IMRInput {
- public static String getTableNameForHCat(TableDesc table) {
- String tableName = (table.isView() || !table.getWholeScan()) ? table.getMaterializedName() : table.getName();
+ public static String getTableNameForHCat(TableDesc table, boolean isPartial) {
+ String tableName = (table.isView() || isPartial) ? table.getMaterializedName() : table.getName();
return String.format("%s.%s", table.getDatabase(), tableName).toUpperCase();
}
@@ -72,7 +72,12 @@ public class HiveMRInput implements IMRInput {
@Override
public IMRTableInputFormat getTableInputFormat(TableDesc table) {
- return new HiveTableInputFormat(getTableNameForHCat(table));
+ return new HiveTableInputFormat(getTableNameForHCat(table, false));
+ }
+
+ @Override
+ public IMRTableInputFormat getTableInputFormat(TableDesc table, boolean isPartial) {
+ return new HiveTableInputFormat(getTableNameForHCat(table, isPartial));
}
@Override
http://git-wip-us.apache.org/repos/asf/kylin/blob/545d1455/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java
index c7987f2..2d5a301 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/KafkaMRInput.java
@@ -68,7 +68,7 @@ public class KafkaMRInput implements IMRInput {
@Override
public IMRBatchCubingInputSide getBatchCubingInputSide(IJoinedFlatTableDesc flatDesc) {
- this.cubeSegment = (CubeSegment)flatDesc.getSegment();
+ this.cubeSegment = (CubeSegment) flatDesc.getSegment();
return new BatchCubingInputSide(cubeSegment);
}
@@ -88,6 +88,11 @@ public class KafkaMRInput implements IMRInput {
}
@Override
+ public IMRTableInputFormat getTableInputFormat(TableDesc table, boolean isPartial) {
+ return getTableInputFormat(table);
+ }
+
+ @Override
public IMRBatchMergeInputSide getBatchMergeInputSide(ISegment seg) {
return new KafkaMRBatchMergeInputSide((CubeSegment) seg);
}