You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by ji...@apache.org on 2015/01/08 17:17:18 UTC
[01/16] tajo git commit: TAJO-1265: min(),
max() does not handle null properly. (Keuntae Park)
Repository: tajo
Updated Branches:
refs/heads/index_support 071c5d05d -> e04c65fdd
TAJO-1265: min(), max() does not handle null properly. (Keuntae Park)
Closes #315
Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/a1e03289
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/a1e03289
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/a1e03289
Branch: refs/heads/index_support
Commit: a1e03289b35d34115a449ab7d81b946f69400210
Parents: 6582d86
Author: Keuntae Park <si...@apache.org>
Authored: Thu Jan 8 17:14:11 2015 +0900
Committer: Keuntae Park <si...@apache.org>
Committed: Thu Jan 8 17:14:11 2015 +0900
----------------------------------------------------------------------
CHANGES | 2 +
.../tajo/engine/function/builtin/AvgDouble.java | 25 +-
.../tajo/engine/function/builtin/AvgFloat.java | 12 +-
.../tajo/engine/function/builtin/AvgInt.java | 14 +-
.../tajo/engine/function/builtin/AvgLong.java | 27 ++-
.../tajo/engine/function/builtin/Max.java | 76 ++++++
.../tajo/engine/function/builtin/MaxDouble.java | 43 +---
.../tajo/engine/function/builtin/MaxFloat.java | 43 +---
.../tajo/engine/function/builtin/MaxInt.java | 44 +---
.../tajo/engine/function/builtin/MaxLong.java | 32 +--
.../tajo/engine/function/builtin/MaxString.java | 52 +----
.../tajo/engine/function/builtin/Min.java | 76 ++++++
.../tajo/engine/function/builtin/MinDouble.java | 41 +---
.../tajo/engine/function/builtin/MinFloat.java | 42 +---
.../tajo/engine/function/builtin/MinInt.java | 43 +---
.../tajo/engine/function/builtin/MinLong.java | 32 +--
.../tajo/engine/function/builtin/MinString.java | 46 +---
.../tajo/engine/function/builtin/SumDouble.java | 34 ++-
.../tajo/engine/function/builtin/SumFloat.java | 33 +--
.../tajo/engine/function/builtin/SumInt.java | 32 +--
.../tajo/engine/function/builtin/SumLong.java | 34 ++-
.../DistinctGroupbySortAggregationExec.java | 41 ++--
.../engine/function/TestBuiltinFunctions.java | 234 +++++++++++++++++++
.../tajo/engine/query/TestGroupByQuery.java | 4 +-
.../testAvgLongOverflow.sql | 1 +
.../testAvgLongOverflow.result | 3 +
.../testGroupByWithNullData2.result | 2 +-
.../testGroupByWithNullData3.result | 2 +-
.../testGroupByWithNullData4.result | 2 +-
.../testGroupByWithNullData6.result | 2 +-
.../testGroupByWithNullData7.result | 2 +-
.../testGroupByWithNullData8.result | 2 +-
.../testLeftOuterJoinWithEmptyTable2.result | 10 +-
.../testLeftOuterJoinWithEmptyTable4.result | 2 +-
.../testLeftOuterJoinWithEmptyTable2.result | 10 +-
.../testLeftOuterJoinWithEmptyTable4.result | 2 +-
.../testLeftOuterJoinWithEmptyTable5.result | 4 +-
37 files changed, 618 insertions(+), 488 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index fe777a7..7783db8 100644
--- a/CHANGES
+++ b/CHANGES
@@ -146,6 +146,8 @@ Release 0.9.1 - unreleased
BUG FIXES
+ TAJO-1265: min(), max() does not handle null properly. (Keuntae Park)
+
TAJO-1270: Fix typos. (DaeMyung Kang via hyunsik)
TAJO-1180: digitValue should throw Exception when char is not in
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
index f337c36..a69beca 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
@@ -41,13 +41,17 @@ import static org.apache.tajo.InternalTypes.AvgDoubleProto;
returnType = Type.FLOAT8,
paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT8})}
)
-public class AvgDouble extends AggFunction {
+public class AvgDouble extends AggFunction<Datum> {
public AvgDouble() {
super(new Column[] {
new Column("expr", Type.FLOAT8)
});
}
+ public AvgDouble(Column[] definedArgs) {
+ super(definedArgs);
+ }
+
public AvgContext newContext() {
return new AvgContext();
}
@@ -55,8 +59,11 @@ public class AvgDouble extends AggFunction {
@Override
public void eval(FunctionContext ctx, Tuple params) {
AvgContext avgCtx = (AvgContext) ctx;
- avgCtx.sum += params.get(0).asFloat8();
- avgCtx.count++;
+ Datum datum = params.get(0);
+ if (datum.isNotNull()) {
+ avgCtx.sum += datum.asFloat8();
+ avgCtx.count++;
+ }
}
@Override
@@ -75,6 +82,9 @@ public class AvgDouble extends AggFunction {
@Override
public Datum getPartialResult(FunctionContext ctx) {
AvgContext avgCtx = (AvgContext) ctx;
+ if (avgCtx.count == 0) {
+ return NullDatum.get();
+ }
AvgDoubleProto.Builder builder = AvgDoubleProto.newBuilder();
builder.setSum(avgCtx.sum);
builder.setCount(avgCtx.count);
@@ -89,11 +99,14 @@ public class AvgDouble extends AggFunction {
@Override
public Datum terminate(FunctionContext ctx) {
AvgContext avgCtx = (AvgContext) ctx;
+ if (avgCtx.count == 0) {
+ return NullDatum.get();
+ }
return DatumFactory.createFloat8(avgCtx.sum / avgCtx.count);
}
protected class AvgContext implements FunctionContext {
- double sum;
- long count;
+ double sum = 0.0;
+ long count = 0;
}
-}
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgFloat.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgFloat.java
index 2370421..8162319 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgFloat.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgFloat.java
@@ -18,7 +18,9 @@
package org.apache.tajo.engine.function.builtin;
+import org.apache.tajo.catalog.Column;
import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.Datum;
import org.apache.tajo.plan.function.FunctionContext;
import org.apache.tajo.engine.function.annotation.Description;
import org.apache.tajo.engine.function.annotation.ParamTypes;
@@ -34,13 +36,19 @@ import org.apache.tajo.storage.Tuple;
public class AvgFloat extends AvgDouble {
public AvgFloat() {
+ super(new Column[] {
+ new Column("expr", TajoDataTypes.Type.FLOAT4)
+ });
}
@Override
public void eval(FunctionContext ctx, Tuple params) {
AvgContext avgCtx = (AvgContext) ctx;
- avgCtx.sum += params.get(0).asFloat4();
- avgCtx.count++;
+ Datum datum = params.get(0);
+ if (datum.isNotNull()) {
+ avgCtx.sum += datum.asFloat4();
+ avgCtx.count++;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgInt.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgInt.java
index 07cf373..1950fb1 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgInt.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgInt.java
@@ -18,7 +18,9 @@
package org.apache.tajo.engine.function.builtin;
+import org.apache.tajo.catalog.Column;
import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.Datum;
import org.apache.tajo.plan.function.FunctionContext;
import org.apache.tajo.engine.function.annotation.Description;
import org.apache.tajo.engine.function.annotation.ParamTypes;
@@ -34,13 +36,19 @@ import org.apache.tajo.storage.Tuple;
public class AvgInt extends AvgLong {
public AvgInt() {
- super();
+ super(new Column[] {
+ new Column("expr", TajoDataTypes.Type.INT4)
+ });
+
}
@Override
public void eval(FunctionContext ctx, Tuple params) {
AvgContext avgCtx = (AvgContext) ctx;
- avgCtx.sum += params.get(0).asInt4();
- avgCtx.count++;
+ Datum datum = params.get(0);
+ if (datum.isNotNull()) {
+ avgCtx.sum += datum.asInt4();
+ avgCtx.count++;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgLong.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgLong.java
index f48de6c..417ecb7 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgLong.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/AvgLong.java
@@ -38,14 +38,18 @@ import static org.apache.tajo.InternalTypes.AvgLongProto;
returnType = Type.FLOAT8,
paramTypes = {@ParamTypes(paramTypes = {Type.INT8})}
)
-public class AvgLong extends AggFunction<Float8Datum> {
+public class AvgLong extends AggFunction<Datum> {
public AvgLong() {
super(new Column[] {
- new Column("expr", Type.FLOAT8)
+ new Column("expr", Type.INT8)
});
}
+ public AvgLong(Column[] definedArgs) {
+ super(definedArgs);
+ }
+
public AvgContext newContext() {
return new AvgContext();
}
@@ -53,8 +57,11 @@ public class AvgLong extends AggFunction<Float8Datum> {
@Override
public void eval(FunctionContext ctx, Tuple params) {
AvgContext avgCtx = (AvgContext) ctx;
- avgCtx.sum += params.get(0).asInt8();
- avgCtx.count++;
+ Datum datum = params.get(0);
+ if (datum.isNotNull()) {
+ avgCtx.sum += datum.asInt8();
+ avgCtx.count++;
+ }
}
@Override
@@ -73,6 +80,9 @@ public class AvgLong extends AggFunction<Float8Datum> {
@Override
public Datum getPartialResult(FunctionContext ctx) {
AvgContext avgCtx = (AvgContext) ctx;
+ if (avgCtx.count == 0) {
+ return NullDatum.get();
+ }
AvgLongProto.Builder builder = AvgLongProto.newBuilder();
builder.setSum(avgCtx.sum);
builder.setCount(avgCtx.count);
@@ -85,13 +95,16 @@ public class AvgLong extends AggFunction<Float8Datum> {
}
@Override
- public Float8Datum terminate(FunctionContext ctx) {
+ public Datum terminate(FunctionContext ctx) {
AvgContext avgCtx = (AvgContext) ctx;
+ if (avgCtx.count == 0) {
+ return NullDatum.get();
+ }
return DatumFactory.createFloat8((double) avgCtx.sum / avgCtx.count);
}
protected class AvgContext implements FunctionContext {
- long sum;
- long count;
+ long sum = 0;
+ long count = 0;
}
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Max.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Max.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Max.java
new file mode 100644
index 0000000..f851994
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Max.java
@@ -0,0 +1,76 @@
+/**
+ * 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.tajo.engine.function.builtin;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.plan.function.AggFunction;
+import org.apache.tajo.plan.function.FunctionContext;
+import org.apache.tajo.storage.Tuple;
+
+public abstract class Max extends AggFunction<Datum> {
+ public Max(Column[] definedArgs) {
+ super(definedArgs);
+ }
+
+ @Override
+ public FunctionContext newContext() {
+ return new MaxContext();
+ }
+
+ @Override
+ public void eval(FunctionContext ctx, Tuple params) {
+ MaxContext maxCtx = (MaxContext) ctx;
+ Datum datum = params.get(0);
+ if (datum.isNotNull()) {
+ if (maxCtx.max == null || maxCtx.max.compareTo(datum) < 0) {
+ maxCtx.max = datum;
+ }
+ }
+ }
+
+ @Override
+ public Datum getPartialResult(FunctionContext ctx) {
+ Datum max = ((MaxContext)ctx).max;
+
+ if (max == null) {
+ return NullDatum.get();
+ }
+ else {
+ return max;
+ }
+ }
+
+ @Override
+ public Datum terminate(FunctionContext ctx) {
+ Datum max = ((MaxContext)ctx).max;
+
+ if (max == null) {
+ return NullDatum.get();
+ }
+ else {
+ return max;
+ }
+ }
+
+ private class MaxContext implements FunctionContext {
+ Datum max = null;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxDouble.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxDouble.java
index a6840c4..c5f5f41 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxDouble.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxDouble.java
@@ -22,24 +22,17 @@ import org.apache.tajo.catalog.CatalogUtil;
import org.apache.tajo.catalog.Column;
import org.apache.tajo.common.TajoDataTypes.DataType;
import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.Float8Datum;
-import org.apache.tajo.plan.function.AggFunction;
-import org.apache.tajo.plan.function.FunctionContext;
import org.apache.tajo.engine.function.annotation.Description;
import org.apache.tajo.engine.function.annotation.ParamTypes;
-import org.apache.tajo.storage.Tuple;
@Description(
- functionName = "max",
- description = "the maximum value of expr",
- example = "> SELECT max(expr);",
- returnType = Type.FLOAT8,
- paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT8})}
+ functionName = "max",
+ description = "the maximum value of expr",
+ example = "> SELECT max(expr);",
+ returnType = Type.FLOAT8,
+ paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT8})}
)
-public class MaxDouble extends AggFunction<Float8Datum> {
-
+public class MaxDouble extends Max {
public MaxDouble() {
super(new Column[] {
new Column("expr", Type.FLOAT8)
@@ -47,32 +40,8 @@ public class MaxDouble extends AggFunction<Float8Datum> {
}
@Override
- public FunctionContext newContext() {
- return new MaxContext();
- }
-
- @Override
- public void eval(FunctionContext ctx, Tuple params) {
- MaxContext maxCtx = (MaxContext) ctx;
- maxCtx.max = Math.max(maxCtx.max, params.get(0).asFloat8());
- }
-
- @Override
- public Datum getPartialResult(FunctionContext ctx) {
- return DatumFactory.createFloat8(((MaxContext) ctx).max);
- }
-
- @Override
public DataType getPartialResultType() {
return CatalogUtil.newSimpleDataType(Type.FLOAT8);
}
- @Override
- public Float8Datum terminate(FunctionContext ctx) {
- return DatumFactory.createFloat8(((MaxContext) ctx).max);
- }
-
- private class MaxContext implements FunctionContext {
- double max;
- }
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxFloat.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxFloat.java
index 6b725af..85fa855 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxFloat.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxFloat.java
@@ -22,55 +22,26 @@ import org.apache.tajo.catalog.CatalogUtil;
import org.apache.tajo.catalog.Column;
import org.apache.tajo.common.TajoDataTypes.DataType;
import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.plan.function.AggFunction;
-import org.apache.tajo.plan.function.FunctionContext;
import org.apache.tajo.engine.function.annotation.Description;
import org.apache.tajo.engine.function.annotation.ParamTypes;
-import org.apache.tajo.storage.Tuple;
@Description(
- functionName = "max",
- description = "the maximum value of expr",
- example = "> SELECT max(expr);",
- returnType = Type.FLOAT4,
- paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT4})}
+ functionName = "max",
+ description = "the maximum value of expr",
+ example = "> SELECT max(expr);",
+ returnType = Type.FLOAT4,
+ paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT4})}
)
-public class MaxFloat extends AggFunction<Datum> {
+public class MaxFloat extends Max {
public MaxFloat() {
super(new Column[] {
- new Column("expr", Type.FLOAT8)
+ new Column("expr", Type.FLOAT4)
});
}
@Override
- public FunctionContext newContext() {
- return new MaxContext();
- }
-
- @Override
- public void eval(FunctionContext ctx, Tuple params) {
- MaxContext maxCtx = (MaxContext) ctx;
- maxCtx.max = Math.max(maxCtx.max, params.get(0).asFloat4());
- }
-
- @Override
- public Datum getPartialResult(FunctionContext ctx) {
- return DatumFactory.createFloat4(((MaxContext) ctx).max);
- }
-
- @Override
public DataType getPartialResultType() {
return CatalogUtil.newSimpleDataType(Type.FLOAT4);
}
- @Override
- public Datum terminate(FunctionContext ctx) {
- return DatumFactory.createFloat4(((MaxContext) ctx).max);
- }
-
- private class MaxContext implements FunctionContext {
- float max;
- }
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxInt.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxInt.java
index 2d9cf48..d879f87 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxInt.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxInt.java
@@ -22,56 +22,26 @@ import org.apache.tajo.catalog.CatalogUtil;
import org.apache.tajo.catalog.Column;
import org.apache.tajo.common.TajoDataTypes.DataType;
import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.plan.function.AggFunction;
-import org.apache.tajo.plan.function.FunctionContext;
import org.apache.tajo.engine.function.annotation.Description;
import org.apache.tajo.engine.function.annotation.ParamTypes;
-import org.apache.tajo.storage.Tuple;
@Description(
- functionName = "max",
- description = "the maximum value of expr",
- example = "> SELECT max(expr);",
- returnType = Type.INT4,
- paramTypes = {@ParamTypes(paramTypes = {Type.INT4})}
+ functionName = "max",
+ description = "the maximum value of expr",
+ example = "> SELECT max(expr);",
+ returnType = Type.INT4,
+ paramTypes = {@ParamTypes(paramTypes = {Type.INT4})}
)
-public class MaxInt extends AggFunction<Datum> {
-
+public class MaxInt extends Max {
public MaxInt() {
super(new Column[] {
- new Column("expr", Type.INT8)
+ new Column("expr", Type.INT4)
});
}
@Override
- public FunctionContext newContext() {
- return new MaxContext();
- }
-
- @Override
- public void eval(FunctionContext ctx, Tuple params) {
- MaxContext maxCtx = (MaxContext) ctx;
- maxCtx.max = Math.max(maxCtx.max, params.get(0).asInt4());
- }
-
- @Override
- public Datum getPartialResult(FunctionContext ctx) {
- return DatumFactory.createInt4(((MaxContext) ctx).max);
- }
-
- @Override
public DataType getPartialResultType() {
return CatalogUtil.newSimpleDataType(Type.INT4);
}
- @Override
- public Datum terminate(FunctionContext ctx) {
- return DatumFactory.createInt4(((MaxContext) ctx).max);
- }
-
- private class MaxContext implements FunctionContext {
- int max;
- }
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxLong.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxLong.java
index b1cc30b..904eca5 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxLong.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxLong.java
@@ -22,14 +22,8 @@ import org.apache.tajo.catalog.CatalogUtil;
import org.apache.tajo.catalog.Column;
import org.apache.tajo.common.TajoDataTypes.DataType;
import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.Int8Datum;
-import org.apache.tajo.plan.function.AggFunction;
-import org.apache.tajo.plan.function.FunctionContext;
import org.apache.tajo.engine.function.annotation.Description;
import org.apache.tajo.engine.function.annotation.ParamTypes;
-import org.apache.tajo.storage.Tuple;
@Description(
functionName = "max",
@@ -38,7 +32,7 @@ import org.apache.tajo.storage.Tuple;
returnType = Type.INT8,
paramTypes = {@ParamTypes(paramTypes = {Type.INT8})}
)
-public class MaxLong extends AggFunction<Int8Datum> {
+public class MaxLong extends Max {
public MaxLong() {
super(new Column[] {
new Column("expr", Type.INT8)
@@ -46,32 +40,8 @@ public class MaxLong extends AggFunction<Int8Datum> {
}
@Override
- public FunctionContext newContext() {
- return new MaxContext();
- }
-
- @Override
- public void eval(FunctionContext ctx, Tuple params) {
- MaxContext maxCtx = (MaxContext) ctx;
- maxCtx.max = Math.max(maxCtx.max, params.get(0).asInt8());
- }
-
- @Override
- public Datum getPartialResult(FunctionContext ctx) {
- return DatumFactory.createInt8(((MaxContext) ctx).max);
- }
-
- @Override
public DataType getPartialResultType() {
return CatalogUtil.newSimpleDataType(Type.INT8);
}
- @Override
- public Int8Datum terminate(FunctionContext ctx) {
- return DatumFactory.createInt8(((MaxContext) ctx).max);
- }
-
- private class MaxContext implements FunctionContext {
- long max;
- }
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxString.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxString.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxString.java
index d3c67ce..239c8b6 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxString.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MaxString.java
@@ -20,62 +20,28 @@ package org.apache.tajo.engine.function.builtin;
import org.apache.tajo.catalog.CatalogUtil;
import org.apache.tajo.catalog.Column;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.TextDatum;
-import org.apache.tajo.plan.function.AggFunction;
-import org.apache.tajo.plan.function.FunctionContext;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
import org.apache.tajo.engine.function.annotation.Description;
import org.apache.tajo.engine.function.annotation.ParamTypes;
-import org.apache.tajo.storage.Tuple;
@Description(
functionName = "max",
description = "the maximum value of expr",
example = "> SELECT max(expr);",
- returnType = TajoDataTypes.Type.TEXT,
- paramTypes = {@ParamTypes(paramTypes = {TajoDataTypes.Type.TEXT})}
+ returnType = Type.TEXT,
+ paramTypes = {@ParamTypes(paramTypes = {Type.TEXT})}
)
-public class MaxString extends AggFunction<Datum> {
-
+public class MaxString extends Max {
public MaxString() {
super(new Column[] {
- new Column("expr", TajoDataTypes.Type.TEXT)
+ new Column("expr", Type.TEXT)
});
}
@Override
- public FunctionContext newContext() {
- return new MaxContext();
- }
-
- @Override
- public void eval(FunctionContext ctx, Tuple params) {
- MaxContext maxCtx = (MaxContext) ctx;
- if (maxCtx.max == null) {
- maxCtx.max = params.get(0).asChars();
- } else if (params.get(0).asChars().compareTo(maxCtx.max) > 0) {
- maxCtx.max = params.get(0).asChars();
- }
- }
-
- @Override
- public Datum getPartialResult(FunctionContext ctx) {
- return DatumFactory.createText(((MaxContext) ctx).max);
+ public DataType getPartialResultType() {
+ return CatalogUtil.newSimpleDataType(Type.TEXT);
}
- @Override
- public TajoDataTypes.DataType getPartialResultType() {
- return CatalogUtil.newSimpleDataType(TajoDataTypes.Type.TEXT);
- }
-
- @Override
- public TextDatum terminate(FunctionContext ctx) {
- return DatumFactory.createText(((MaxContext) ctx).max);
- }
-
- private class MaxContext implements FunctionContext {
- String max;
- }
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Min.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Min.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Min.java
new file mode 100644
index 0000000..2695b6a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Min.java
@@ -0,0 +1,76 @@
+/**
+ * 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.tajo.engine.function.builtin;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.plan.function.AggFunction;
+import org.apache.tajo.plan.function.FunctionContext;
+import org.apache.tajo.storage.Tuple;
+
+public abstract class Min extends AggFunction<Datum> {
+ public Min(Column[] definedArgs) {
+ super(definedArgs);
+ }
+
+ @Override
+ public FunctionContext newContext() {
+ return new MinContext();
+ }
+
+ @Override
+ public void eval(FunctionContext ctx, Tuple params) {
+ MinContext minCtx = (MinContext) ctx;
+ Datum datum = params.get(0);
+ if (datum.isNotNull()) {
+ if (minCtx.min == null || minCtx.min.compareTo(datum) > 0) {
+ minCtx.min = datum;
+ }
+ }
+ }
+
+ @Override
+ public Datum getPartialResult(FunctionContext ctx) {
+ Datum min = ((MinContext)ctx).min;
+
+ if (min == null) {
+ return NullDatum.get();
+ }
+ else {
+ return min;
+ }
+ }
+
+ @Override
+ public Datum terminate(FunctionContext ctx) {
+ Datum min = ((MinContext)ctx).min;
+
+ if (min == null) {
+ return NullDatum.get();
+ }
+ else {
+ return min;
+ }
+ }
+
+ private class MinContext implements FunctionContext {
+ Datum min = null;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinDouble.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinDouble.java
index 38eba48..8999f0a 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinDouble.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinDouble.java
@@ -22,22 +22,17 @@ import org.apache.tajo.catalog.CatalogUtil;
import org.apache.tajo.catalog.Column;
import org.apache.tajo.common.TajoDataTypes.DataType;
import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.plan.function.AggFunction;
-import org.apache.tajo.plan.function.FunctionContext;
import org.apache.tajo.engine.function.annotation.Description;
import org.apache.tajo.engine.function.annotation.ParamTypes;
-import org.apache.tajo.storage.Tuple;
@Description(
- functionName = "min",
- description = "the minimum value of expr",
- example = "> SELECT min(expr);",
- returnType = Type.FLOAT8,
- paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT8})}
+ functionName = "min",
+ description = "the minimum value of expr",
+ example = "> SELECT min(expr);",
+ returnType = Type.FLOAT8,
+ paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT8})}
)
-public class MinDouble extends AggFunction<Datum> {
+public class MinDouble extends Min {
public MinDouble() {
super(new Column[] {
@@ -46,32 +41,8 @@ public class MinDouble extends AggFunction<Datum> {
}
@Override
- public FunctionContext newContext() {
- return new MinContext();
- }
-
- @Override
- public void eval(FunctionContext ctx, Tuple params) {
- MinContext minCtx = (MinContext) ctx;
- minCtx.min = Math.min(minCtx.min, params.get(0).asFloat8());
- }
-
- @Override
- public Datum getPartialResult(FunctionContext ctx) {
- return DatumFactory.createFloat8(((MinContext) ctx).min);
- }
-
- @Override
public DataType getPartialResultType() {
return CatalogUtil.newSimpleDataType(Type.FLOAT8);
}
- @Override
- public Datum terminate(FunctionContext ctx) {
- return DatumFactory.createFloat8(((MinContext) ctx).min);
- }
-
- private class MinContext implements FunctionContext {
- double min = Double.MAX_VALUE;
- }
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinFloat.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinFloat.java
index 17e63e2..e064f28 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinFloat.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinFloat.java
@@ -22,23 +22,17 @@ import org.apache.tajo.catalog.CatalogUtil;
import org.apache.tajo.catalog.Column;
import org.apache.tajo.common.TajoDataTypes.DataType;
import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.Float4Datum;
-import org.apache.tajo.plan.function.AggFunction;
-import org.apache.tajo.plan.function.FunctionContext;
import org.apache.tajo.engine.function.annotation.Description;
import org.apache.tajo.engine.function.annotation.ParamTypes;
-import org.apache.tajo.storage.Tuple;
@Description(
- functionName = "min",
- description = "the minimum value of expr",
- example = "> SELECT min(expr);",
- returnType = Type.FLOAT4,
- paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT4})}
+ functionName = "min",
+ description = "the minimum value of expr",
+ example = "> SELECT min(expr);",
+ returnType = Type.FLOAT4,
+ paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT4})}
)
-public class MinFloat extends AggFunction<Float4Datum> {
+public class MinFloat extends Min {
public MinFloat() {
super(new Column[] {
@@ -47,32 +41,8 @@ public class MinFloat extends AggFunction<Float4Datum> {
}
@Override
- public FunctionContext newContext() {
- return new MinContext();
- }
-
- @Override
- public void eval(FunctionContext ctx, Tuple params) {
- MinContext minCtx = (MinContext) ctx;
- minCtx.min = Math.min(minCtx.min, params.get(0).asFloat4());
- }
-
- @Override
- public Datum getPartialResult(FunctionContext ctx) {
- return DatumFactory.createFloat4(((MinContext) ctx).min);
- }
-
- @Override
public DataType getPartialResultType() {
return CatalogUtil.newSimpleDataType(Type.FLOAT4);
}
- @Override
- public Float4Datum terminate(FunctionContext ctx) {
- return DatumFactory.createFloat4(((MinContext) ctx).min);
- }
-
- private class MinContext implements FunctionContext {
- float min = Float.MAX_VALUE;
- }
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinInt.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinInt.java
index d2a5a2d..4b66309 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinInt.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinInt.java
@@ -22,56 +22,27 @@ import org.apache.tajo.catalog.CatalogUtil;
import org.apache.tajo.catalog.Column;
import org.apache.tajo.common.TajoDataTypes.DataType;
import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.plan.function.AggFunction;
-import org.apache.tajo.plan.function.FunctionContext;
import org.apache.tajo.engine.function.annotation.Description;
import org.apache.tajo.engine.function.annotation.ParamTypes;
-import org.apache.tajo.storage.Tuple;
@Description(
- functionName = "min",
- description = "the minimum value of expr",
- example = "> SELECT min(expr);",
- returnType = Type.INT4,
- paramTypes = {@ParamTypes(paramTypes = {Type.INT4})}
+ functionName = "min",
+ description = "the minimum value of expr",
+ example = "> SELECT min(expr);",
+ returnType = Type.INT4,
+ paramTypes = {@ParamTypes(paramTypes = {Type.INT4})}
)
-public class MinInt extends AggFunction<Datum> {
+public class MinInt extends Min {
public MinInt() {
super(new Column[] {
- new Column("expr", Type.INT8)
+ new Column("expr", Type.INT4)
});
}
@Override
- public FunctionContext newContext() {
- return new MinContext();
- }
-
- @Override
- public void eval(FunctionContext ctx, Tuple params) {
- MinContext minCtx = (MinContext) ctx;
- minCtx.min = Math.min(minCtx.min, params.get(0).asInt4());
- }
-
- @Override
- public Datum getPartialResult(FunctionContext ctx) {
- return DatumFactory.createInt4(((MinContext) ctx).min);
- }
-
- @Override
public DataType getPartialResultType() {
return CatalogUtil.newSimpleDataType(Type.INT4);
}
- @Override
- public Datum terminate(FunctionContext ctx) {
- return DatumFactory.createInt4(((MinContext) ctx).min);
- }
-
- private class MinContext implements FunctionContext {
- int min = Integer.MAX_VALUE;
- }
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinLong.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinLong.java
index e346ac7..5861037 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinLong.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinLong.java
@@ -22,14 +22,8 @@ import org.apache.tajo.catalog.CatalogUtil;
import org.apache.tajo.catalog.Column;
import org.apache.tajo.common.TajoDataTypes.DataType;
import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.Int8Datum;
-import org.apache.tajo.plan.function.AggFunction;
-import org.apache.tajo.plan.function.FunctionContext;
import org.apache.tajo.engine.function.annotation.Description;
import org.apache.tajo.engine.function.annotation.ParamTypes;
-import org.apache.tajo.storage.Tuple;
@Description(
functionName = "min",
@@ -38,7 +32,7 @@ import org.apache.tajo.storage.Tuple;
returnType = Type.INT8,
paramTypes = {@ParamTypes(paramTypes = {Type.INT8})}
)
-public class MinLong extends AggFunction<Datum> {
+public class MinLong extends Min {
public MinLong() {
super(new Column[] {
@@ -47,32 +41,8 @@ public class MinLong extends AggFunction<Datum> {
}
@Override
- public FunctionContext newContext() {
- return new MinContext();
- }
-
- @Override
- public void eval(FunctionContext ctx, Tuple params) {
- MinContext minCtx = (MinContext)ctx;
- minCtx.min = Math.min(minCtx.min, params.get(0).asInt8());
- }
-
- @Override
- public Datum getPartialResult(FunctionContext ctx) {
- return DatumFactory.createInt8(((MinContext) ctx).min);
- }
-
- @Override
public DataType getPartialResultType() {
return CatalogUtil.newSimpleDataType(Type.INT8);
}
- @Override
- public Int8Datum terminate(FunctionContext ctx) {
- return DatumFactory.createInt8(((MinContext) ctx).min);
- }
-
- private class MinContext implements FunctionContext {
- long min = Long.MAX_VALUE;
- }
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinString.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinString.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinString.java
index 7e87a49..3b8e6d5 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinString.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/MinString.java
@@ -22,23 +22,17 @@ import org.apache.tajo.catalog.CatalogUtil;
import org.apache.tajo.catalog.Column;
import org.apache.tajo.common.TajoDataTypes.DataType;
import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.TextDatum;
-import org.apache.tajo.plan.function.AggFunction;
-import org.apache.tajo.plan.function.FunctionContext;
import org.apache.tajo.engine.function.annotation.Description;
import org.apache.tajo.engine.function.annotation.ParamTypes;
-import org.apache.tajo.storage.Tuple;
@Description(
- functionName = "min",
- description = "the minimum value of expr",
- example = "> SELECT min(expr);",
- returnType = Type.TEXT,
- paramTypes = {@ParamTypes(paramTypes = {Type.TEXT})}
+ functionName = "min",
+ description = "the minimum value of expr",
+ example = "> SELECT min(expr);",
+ returnType = Type.TEXT,
+ paramTypes = {@ParamTypes(paramTypes = {Type.TEXT})}
)
-public class MinString extends AggFunction<Datum> {
+public class MinString extends Min {
public MinString() {
super(new Column[] {
@@ -47,36 +41,8 @@ public class MinString extends AggFunction<Datum> {
}
@Override
- public FunctionContext newContext() {
- return new MinContext();
- }
-
- @Override
- public void eval(FunctionContext ctx, Tuple params) {
- MinContext minCtx = (MinContext) ctx;
- if (minCtx.min == null) {
- minCtx.min = params.get(0).asChars();
- } else if (params.get(0).asChars().compareTo(minCtx.min) < 0) {
- minCtx.min = params.get(0).asChars();
- }
- }
-
- @Override
- public Datum getPartialResult(FunctionContext ctx) {
- return DatumFactory.createText(((MinContext) ctx).min);
- }
-
- @Override
public DataType getPartialResultType() {
return CatalogUtil.newSimpleDataType(Type.TEXT);
}
- @Override
- public TextDatum terminate(FunctionContext ctx) {
- return DatumFactory.createText(((MinContext) ctx).min);
- }
-
- private class MinContext implements FunctionContext {
- String min;
- }
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumDouble.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumDouble.java
index a2da84d..2f42272 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumDouble.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumDouble.java
@@ -24,7 +24,7 @@ import org.apache.tajo.common.TajoDataTypes.DataType;
import org.apache.tajo.common.TajoDataTypes.Type;
import org.apache.tajo.datum.Datum;
import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.Float8Datum;
+import org.apache.tajo.datum.NullDatum;
import org.apache.tajo.plan.function.AggFunction;
import org.apache.tajo.plan.function.FunctionContext;
import org.apache.tajo.engine.function.annotation.Description;
@@ -51,6 +51,10 @@ public class SumDouble extends AggFunction<Datum> {
});
}
+ public SumDouble(Column[] definedArgs) {
+ super(definedArgs);
+ }
+
@Override
public FunctionContext newContext() {
return new SumContext();
@@ -58,12 +62,22 @@ public class SumDouble extends AggFunction<Datum> {
@Override
public void eval(FunctionContext ctx, Tuple params) {
- ((SumContext)ctx).sum += params.get(0).asFloat8();
+ Datum datum = params.get(0);
+ if (datum.isNotNull()) {
+ SumContext sumCtx = (SumContext)ctx;
+ sumCtx.hasNonNull = true;
+ sumCtx.sum += datum.asFloat8();
+ }
}
@Override
public Datum getPartialResult(FunctionContext ctx) {
- return DatumFactory.createFloat8(((SumContext) ctx).sum);
+ SumContext sumCtx = (SumContext)ctx;
+ if (sumCtx.hasNonNull) {
+ return DatumFactory.createFloat8(sumCtx.sum);
+ } else {
+ return NullDatum.get();
+ }
}
@Override
@@ -72,11 +86,17 @@ public class SumDouble extends AggFunction<Datum> {
}
@Override
- public Float8Datum terminate(FunctionContext ctx) {
- return DatumFactory.createFloat8(((SumContext) ctx).sum);
+ public Datum terminate(FunctionContext ctx) {
+ SumContext sumCtx = (SumContext)ctx;
+ if (sumCtx.hasNonNull) {
+ return DatumFactory.createFloat8(sumCtx.sum);
+ } else {
+ return NullDatum.get();
+ }
}
- private class SumContext implements FunctionContext {
- double sum;
+ protected class SumContext implements FunctionContext {
+ boolean hasNonNull = false;
+ double sum = 0.0;
}
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumFloat.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumFloat.java
index 73257e2..f63aa56 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumFloat.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumFloat.java
@@ -42,7 +42,7 @@ import org.apache.tajo.storage.Tuple;
returnType = Type.FLOAT8,
paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT4})}
)
-public class SumFloat extends AggFunction<Datum> {
+public class SumFloat extends SumDouble {
public SumFloat() {
super(new Column[] {
new Column("expr", Type.FLOAT4)
@@ -50,31 +50,12 @@ public class SumFloat extends AggFunction<Datum> {
}
@Override
- public FunctionContext newContext() {
- return new SumContext();
- }
-
- @Override
public void eval(FunctionContext ctx, Tuple params) {
- ((SumContext)ctx).sum += params.get(0).asFloat4();
- }
-
- @Override
- public Datum getPartialResult(FunctionContext ctx) {
- return DatumFactory.createFloat8(((SumContext) ctx).sum);
- }
-
- @Override
- public DataType getPartialResultType() {
- return CatalogUtil.newSimpleDataType(Type.FLOAT8);
- }
-
- @Override
- public Datum terminate(FunctionContext ctx) {
- return DatumFactory.createFloat8(((SumContext) ctx).sum);
- }
-
- private class SumContext implements FunctionContext {
- private double sum;
+ Datum datum = params.get(0);
+ if (datum.isNotNull()) {
+ SumContext sumCtx = (SumContext)ctx;
+ sumCtx.hasNonNull = true;
+ sumCtx.sum += datum.asFloat4();
+ }
}
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumInt.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumInt.java
index da8bf16..5f68e21 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumInt.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumInt.java
@@ -42,41 +42,11 @@ import org.apache.tajo.storage.Tuple;
returnType = Type.INT8,
paramTypes = {@ParamTypes(paramTypes = {Type.INT4})}
)
-public class SumInt extends AggFunction<Datum> {
+public class SumInt extends SumLong {
public SumInt() {
super(new Column[] {
new Column("expr", Type.INT4)
});
}
-
- @Override
- public SumIntContext newContext() {
- return new SumIntContext();
- }
-
- @Override
- public void eval(FunctionContext ctx, Tuple params) {
- SumIntContext sumCtx = (SumIntContext) ctx;
- sumCtx.sum += params.get(0).asInt8();
- }
-
- @Override
- public Datum getPartialResult(FunctionContext ctx) {
- return DatumFactory.createInt8(((SumIntContext) ctx).sum);
- }
-
- @Override
- public DataType getPartialResultType() {
- return CatalogUtil.newSimpleDataType(Type.INT8);
- }
-
- @Override
- public Datum terminate(FunctionContext ctx) {
- return DatumFactory.createInt8(((SumIntContext) ctx).sum);
- }
-
- private class SumIntContext implements FunctionContext {
- long sum;
- }
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumLong.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumLong.java
index 7a6e707..55484d7 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumLong.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/SumLong.java
@@ -24,7 +24,7 @@ import org.apache.tajo.common.TajoDataTypes.DataType;
import org.apache.tajo.common.TajoDataTypes.Type;
import org.apache.tajo.datum.Datum;
import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.datum.Int8Datum;
+import org.apache.tajo.datum.NullDatum;
import org.apache.tajo.plan.function.AggFunction;
import org.apache.tajo.plan.function.FunctionContext;
import org.apache.tajo.engine.function.annotation.Description;
@@ -51,6 +51,10 @@ public class SumLong extends AggFunction<Datum> {
});
}
+ public SumLong(Column[] definedArgs) {
+ super(definedArgs);
+ }
+
@Override
public FunctionContext newContext() {
return new SumContext();
@@ -58,12 +62,22 @@ public class SumLong extends AggFunction<Datum> {
@Override
public void eval(FunctionContext ctx, Tuple params) {
- ((SumContext)ctx).sum += params.get(0).asInt8();
+ Datum datum = params.get(0);
+ if (datum.isNotNull()) {
+ SumContext sumCtx = (SumContext) ctx;
+ sumCtx.hasNonNull = true;
+ sumCtx.sum += datum.asInt8();
+ }
}
@Override
public Datum getPartialResult(FunctionContext ctx) {
- return DatumFactory.createInt8(((SumContext) ctx).sum);
+ SumContext sumCtx = (SumContext) ctx;
+ if (sumCtx.hasNonNull) {
+ return DatumFactory.createInt8(sumCtx.sum);
+ } else {
+ return NullDatum.get();
+ }
}
@Override
@@ -72,11 +86,17 @@ public class SumLong extends AggFunction<Datum> {
}
@Override
- public Int8Datum terminate(FunctionContext ctx) {
- return DatumFactory.createInt8(((SumContext) ctx).sum);
+ public Datum terminate(FunctionContext ctx) {
+ SumContext sumCtx = (SumContext) ctx;
+ if (sumCtx.hasNonNull) {
+ return DatumFactory.createInt8(sumCtx.sum);
+ } else {
+ return NullDatum.get();
+ }
}
- private class SumContext implements FunctionContext {
- long sum;
+ protected class SumContext implements FunctionContext {
+ boolean hasNonNull;
+ long sum = 0;
}
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySortAggregationExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySortAggregationExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySortAggregationExec.java
index 6641633..3a84f98 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySortAggregationExec.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySortAggregationExec.java
@@ -22,6 +22,7 @@ import org.apache.tajo.catalog.statistics.TableStats;
import org.apache.tajo.common.TajoDataTypes;
import org.apache.tajo.datum.DatumFactory;
import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.plan.expr.AggregationFunctionCallEval;
import org.apache.tajo.plan.logical.DistinctGroupbyNode;
import org.apache.tajo.plan.logical.GroupbyNode;
import org.apache.tajo.storage.Tuple;
@@ -127,23 +128,33 @@ public class DistinctGroupbySortAggregationExec extends PhysicalExec {
}
private Tuple getEmptyTuple() {
- Tuple tuple = new VTuple(outColumnNum);
+ Tuple tuple = new VTuple(outSchema.size());
NullDatum nullDatum = DatumFactory.createNullDatum();
- for (int i = 0; i < outColumnNum; i++) {
- TajoDataTypes.Type type = outSchema.getColumn(i).getDataType().getType();
- if (type == TajoDataTypes.Type.INT8) {
- tuple.put(i, DatumFactory.createInt8(nullDatum.asInt8()));
- } else if (type == TajoDataTypes.Type.INT4) {
- tuple.put(i, DatumFactory.createInt4(nullDatum.asInt4()));
- } else if (type == TajoDataTypes.Type.INT2) {
- tuple.put(i, DatumFactory.createInt2(nullDatum.asInt2()));
- } else if (type == TajoDataTypes.Type.FLOAT4) {
- tuple.put(i, DatumFactory.createFloat4(nullDatum.asFloat4()));
- } else if (type == TajoDataTypes.Type.FLOAT8) {
- tuple.put(i, DatumFactory.createFloat8(nullDatum.asFloat8()));
- } else {
- tuple.put(i, DatumFactory.createNullDatum());
+ int tupleIndex = 0;
+ for (SortAggregateExec aggExec: aggregateExecs) {
+ for (int i = 0; i < aggExec.aggFunctionsNum; i++, tupleIndex++) {
+ String funcName = aggExec.aggFunctions[i].getName();
+ if ("min".equals(funcName) || "max".equals(funcName) || "avg".equals(funcName) || "sum".equals(funcName)) {
+ tuple.put(resultColumnIdIndexes[tupleIndex], DatumFactory.createNullDatum());
+ }
+ else
+ {
+ TajoDataTypes.Type type = outSchema.getColumn(resultColumnIdIndexes[tupleIndex]).getDataType().getType();
+ if (type == TajoDataTypes.Type.INT8) {
+ tuple.put(resultColumnIdIndexes[tupleIndex], DatumFactory.createInt8(nullDatum.asInt8()));
+ } else if (type == TajoDataTypes.Type.INT4) {
+ tuple.put(resultColumnIdIndexes[tupleIndex], DatumFactory.createInt4(nullDatum.asInt4()));
+ } else if (type == TajoDataTypes.Type.INT2) {
+ tuple.put(resultColumnIdIndexes[tupleIndex], DatumFactory.createInt2(nullDatum.asInt2()));
+ } else if (type == TajoDataTypes.Type.FLOAT4) {
+ tuple.put(resultColumnIdIndexes[tupleIndex], DatumFactory.createFloat4(nullDatum.asFloat4()));
+ } else if (type == TajoDataTypes.Type.FLOAT8) {
+ tuple.put(resultColumnIdIndexes[tupleIndex], DatumFactory.createFloat8(nullDatum.asFloat8()));
+ } else {
+ tuple.put(resultColumnIdIndexes[tupleIndex], DatumFactory.createNullDatum());
+ }
+ }
}
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java
index 4fefe07..4578ae5 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java
@@ -21,11 +21,18 @@ package org.apache.tajo.engine.function;
import org.apache.tajo.IntegrationTest;
import org.apache.tajo.QueryTestCaseBase;
import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.util.KeyValueSet;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import java.sql.ResultSet;
+import static org.junit.Assert.assertEquals;
+
@Category(IntegrationTest.class)
public class TestBuiltinFunctions extends QueryTestCaseBase {
@@ -41,6 +48,32 @@ public class TestBuiltinFunctions extends QueryTestCaseBase {
}
@Test
+ public void testMaxLongWithNull() throws Exception {
+ KeyValueSet tableOptions = new KeyValueSet();
+ tableOptions.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+ tableOptions.set(StorageConstants.TEXT_NULL, "\\\\N");
+
+ Schema schema = new Schema();
+ schema.addColumn("id", TajoDataTypes.Type.INT4);
+ schema.addColumn("value", TajoDataTypes.Type.INT8);
+ String[] data = new String[]{ "1|-111", "2|\\N", "3|-333" };
+ TajoTestingCluster.createTable("table11", schema, tableOptions, data, 1);
+
+ try {
+ ResultSet res = executeString("select max(value) as max_value from table11");
+ String ascExpected = "max_value\n" +
+ "-------------------------------\n" +
+ "-111\n";
+
+ assertEquals(ascExpected, resultSetToString(res));
+ res.close();
+ } finally {
+ executeString("DROP TABLE table11 PURGE");
+ }
+
+ }
+
+ @Test
public void testMinLong() throws Exception {
ResultSet res = executeQuery();
assertResultSet(res);
@@ -48,6 +81,32 @@ public class TestBuiltinFunctions extends QueryTestCaseBase {
}
@Test
+ public void testMinLongWithNull() throws Exception {
+ KeyValueSet tableOptions = new KeyValueSet();
+ tableOptions.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+ tableOptions.set(StorageConstants.TEXT_NULL, "\\\\N");
+
+ Schema schema = new Schema();
+ schema.addColumn("id", TajoDataTypes.Type.INT4);
+ schema.addColumn("value", TajoDataTypes.Type.INT8);
+ String[] data = new String[]{ "1|111", "2|\\N", "3|333" };
+ TajoTestingCluster.createTable("table11", schema, tableOptions, data, 1);
+
+ try {
+ ResultSet res = executeString("select min(value) as min_value from table11");
+ String ascExpected = "min_value\n" +
+ "-------------------------------\n" +
+ "111\n";
+
+ assertEquals(ascExpected, resultSetToString(res));
+ res.close();
+ } finally {
+ executeString("DROP TABLE table11 PURGE");
+ }
+
+ }
+
+ @Test
public void testMaxString() throws Exception {
ResultSet res = executeQuery();
assertResultSet(res);
@@ -55,6 +114,32 @@ public class TestBuiltinFunctions extends QueryTestCaseBase {
}
@Test
+ public void testMaxStringWithNull() throws Exception {
+ KeyValueSet tableOptions = new KeyValueSet();
+ tableOptions.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+ tableOptions.set(StorageConstants.TEXT_NULL, "\\\\N");
+
+ Schema schema = new Schema();
+ schema.addColumn("id", TajoDataTypes.Type.INT4);
+ schema.addColumn("name", TajoDataTypes.Type.TEXT);
+ String[] data = new String[]{ "1|\\N", "2|\\N", "3|\\N" };
+ TajoTestingCluster.createTable("table11", schema, tableOptions, data, 1);
+
+ try {
+ ResultSet res = executeString("select max(name) as max_name from table11");
+ String ascExpected = "max_name\n" +
+ "-------------------------------\n" +
+ "null\n";
+
+ assertEquals(ascExpected, resultSetToString(res));
+ res.close();
+ } finally {
+ executeString("DROP TABLE table11 PURGE");
+ }
+
+ }
+
+ @Test
public void testMinString() throws Exception {
ResultSet res = executeQuery();
assertResultSet(res);
@@ -62,6 +147,32 @@ public class TestBuiltinFunctions extends QueryTestCaseBase {
}
@Test
+ public void testMinStringWithNull() throws Exception {
+ KeyValueSet tableOptions = new KeyValueSet();
+ tableOptions.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+ tableOptions.set(StorageConstants.TEXT_NULL, "\\\\N");
+
+ Schema schema = new Schema();
+ schema.addColumn("id", TajoDataTypes.Type.INT4);
+ schema.addColumn("name", TajoDataTypes.Type.TEXT);
+ String[] data = new String[]{ "1|def", "2|\\N", "3|abc" };
+ TajoTestingCluster.createTable("table11", schema, tableOptions, data, 1);
+
+ try {
+ ResultSet res = executeString("select min(name) as min_name from table11");
+ String ascExpected = "min_name\n" +
+ "-------------------------------\n" +
+ "abc\n";
+
+ assertEquals(ascExpected, resultSetToString(res));
+ res.close();
+ } finally {
+ executeString("DROP TABLE table11 PURGE");
+ }
+
+ }
+
+ @Test
public void testCount() throws Exception {
ResultSet res = executeQuery();
assertResultSet(res);
@@ -89,6 +200,129 @@ public class TestBuiltinFunctions extends QueryTestCaseBase {
cleanupQuery(res);
}
+ @Test
+ public void testAvgLongOverflow() throws Exception {
+ ResultSet res = executeQuery();
+ assertResultSet(res);
+ cleanupQuery(res);
+ }
+
+ @Test
+ public void testAvgWithNull() throws Exception {
+ KeyValueSet tableOptions = new KeyValueSet();
+ tableOptions.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+ tableOptions.set(StorageConstants.TEXT_NULL, "\\\\N");
+
+ Schema schema = new Schema();
+ schema.addColumn("id", TajoDataTypes.Type.INT4);
+ schema.addColumn("value_int", TajoDataTypes.Type.INT4);
+ schema.addColumn("value_long", TajoDataTypes.Type.INT8);
+ schema.addColumn("value_float", TajoDataTypes.Type.FLOAT4);
+ schema.addColumn("value_double", TajoDataTypes.Type.FLOAT8);
+ String[] data = new String[]{ "1|\\N|-111|1.2|-50.5", "2|1|\\N|\\N|52.5", "3|2|-333|2.8|\\N" };
+ TajoTestingCluster.createTable("table11", schema, tableOptions, data, 1);
+
+ try {
+ ResultSet res = executeString("select avg(value_int) as avg_int, avg(value_long) as avg_long, avg(value_float) as avg_float, avg(value_double) as avg_double from table11");
+ String ascExpected = "avg_int,avg_long,avg_float,avg_double\n" +
+ "-------------------------------\n" +
+ "1.5,-222.0,2.0,1.0\n";
+
+ assertEquals(ascExpected, resultSetToString(res));
+ res.close();
+ } finally {
+ executeString("DROP TABLE table11 PURGE");
+ }
+
+ }
+
+ @Test
+ public void testAvgWithAllNulls() throws Exception {
+ KeyValueSet tableOptions = new KeyValueSet();
+ tableOptions.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+ tableOptions.set(StorageConstants.TEXT_NULL, "\\\\N");
+
+ Schema schema = new Schema();
+ schema.addColumn("id", TajoDataTypes.Type.INT4);
+ schema.addColumn("value_int", TajoDataTypes.Type.INT4);
+ schema.addColumn("value_long", TajoDataTypes.Type.INT8);
+ schema.addColumn("value_float", TajoDataTypes.Type.FLOAT4);
+ schema.addColumn("value_double", TajoDataTypes.Type.FLOAT8);
+ String[] data = new String[]{ "1|\\N|\\N|\\N|\\N", "2|\\N|\\N|\\N|\\N", "3|\\N|\\N|\\N|\\N" };
+ TajoTestingCluster.createTable("table11", schema, tableOptions, data, 1);
+
+ try {
+ ResultSet res = executeString("select avg(value_int) as avg_int, avg(value_long) as avg_long, avg(value_float) as avg_float, avg(value_double) as avg_double from table11");
+ String ascExpected = "avg_int,avg_long,avg_float,avg_double\n" +
+ "-------------------------------\n" +
+ "null,null,null,null\n";
+
+ assertEquals(ascExpected, resultSetToString(res));
+ res.close();
+ } finally {
+ executeString("DROP TABLE table11 PURGE");
+ }
+
+ }
+
+ @Test
+ public void testSumWithNull() throws Exception {
+ KeyValueSet tableOptions = new KeyValueSet();
+ tableOptions.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+ tableOptions.set(StorageConstants.TEXT_NULL, "\\\\N");
+
+ Schema schema = new Schema();
+ schema.addColumn("id", TajoDataTypes.Type.INT4);
+ schema.addColumn("value_int", TajoDataTypes.Type.INT4);
+ schema.addColumn("value_long", TajoDataTypes.Type.INT8);
+ schema.addColumn("value_float", TajoDataTypes.Type.FLOAT4);
+ schema.addColumn("value_double", TajoDataTypes.Type.FLOAT8);
+ String[] data = new String[]{ "1|\\N|-111|1.2|-50.5", "2|1|\\N|\\N|52.5", "3|2|-333|2.8|\\N" };
+ TajoTestingCluster.createTable("table11", schema, tableOptions, data, 1);
+
+ try {
+ ResultSet res = executeString("select sum(value_int) as sum_int, sum(value_long) as sum_long, sum(value_float) as sum_float, sum(value_double) as sum_double from table11");
+ String ascExpected = "sum_int,sum_long,sum_float,sum_double\n" +
+ "-------------------------------\n" +
+ "3,-444,4.0,2.0\n";
+
+ assertEquals(ascExpected, resultSetToString(res));
+ res.close();
+ } finally {
+ executeString("DROP TABLE table11 PURGE");
+ }
+
+ }
+
+ @Test
+ public void testSumWithAllNulls() throws Exception {
+ KeyValueSet tableOptions = new KeyValueSet();
+ tableOptions.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
+ tableOptions.set(StorageConstants.TEXT_NULL, "\\\\N");
+
+ Schema schema = new Schema();
+ schema.addColumn("id", TajoDataTypes.Type.INT4);
+ schema.addColumn("value_int", TajoDataTypes.Type.INT4);
+ schema.addColumn("value_long", TajoDataTypes.Type.INT8);
+ schema.addColumn("value_float", TajoDataTypes.Type.FLOAT4);
+ schema.addColumn("value_double", TajoDataTypes.Type.FLOAT8);
+ String[] data = new String[]{ "1|\\N|\\N|\\N|\\N", "2|\\N|\\N|\\N|\\N", "3|\\N|\\N|\\N|\\N" };
+ TajoTestingCluster.createTable("table11", schema, tableOptions, data, 1);
+
+ try {
+ ResultSet res = executeString("select sum(value_int) as sum_int, sum(value_long) as sum_long, sum(value_float) as sum_float, sum(value_double) as sum_double from table11");
+ String ascExpected = "sum_int,sum_long,sum_float,sum_double\n" +
+ "-------------------------------\n" +
+ "null,null,null,null\n";
+
+ assertEquals(ascExpected, resultSetToString(res));
+ res.close();
+ } finally {
+ executeString("DROP TABLE table11 PURGE");
+ }
+
+ }
+
// @Test
// public void testRandom() throws Exception {
// ResultSet res = executeQuery();
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
index 794c14f..1a212b0 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
@@ -441,8 +441,8 @@ public class TestGroupByQuery extends QueryTestCaseBase {
String expected = "id,?count_4,?avg_5,?min_6,?max_7,?sum_8,?cast_9,?cast_10,?cast_11,?cast_12\n" +
"-------------------------------\n" +
- "1,2,4.0,0,5,12,4,0,5,12\n" +
- "2,3,2.0,0,3,6,7,0,8,21\n";
+ "1,2,4.0,3,5,12,4,3,5,12\n" +
+ "2,3,2.0,1,3,6,7,6,8,21\n";
assertEquals(expected, resultSetToString(res));
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/resources/queries/TestBuiltinFunctions/testAvgLongOverflow.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestBuiltinFunctions/testAvgLongOverflow.sql b/tajo-core/src/test/resources/queries/TestBuiltinFunctions/testAvgLongOverflow.sql
new file mode 100644
index 0000000..51a2ac1
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestBuiltinFunctions/testAvgLongOverflow.sql
@@ -0,0 +1 @@
+select avg(cast(l_quantity * 25264513 as INT4)) as avg from lineitem where l_quantity > 0;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/resources/results/TestBuiltinFunctions/testAvgLongOverflow.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestBuiltinFunctions/testAvgLongOverflow.result b/tajo-core/src/test/resources/results/TestBuiltinFunctions/testAvgLongOverflow.result
new file mode 100644
index 0000000..5e1c937
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestBuiltinFunctions/testAvgLongOverflow.result
@@ -0,0 +1,3 @@
+avg
+-------------------------------
+9.34786981E8
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData2.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData2.result b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData2.result
index f4f9a5b..8746e36 100644
--- a/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData2.result
+++ b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData2.result
@@ -1,3 +1,3 @@
unique_key,max_key
-------------------------------
-0,0
\ No newline at end of file
+0,null
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData3.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData3.result b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData3.result
index fef3d0c..c2cc22e 100644
--- a/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData3.result
+++ b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData3.result
@@ -1,3 +1,3 @@
maximum,unique_key
-------------------------------
-0,0
\ No newline at end of file
+null,0
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData4.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData4.result b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData4.result
index fef3d0c..c2cc22e 100644
--- a/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData4.result
+++ b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData4.result
@@ -1,3 +1,3 @@
maximum,unique_key
-------------------------------
-0,0
\ No newline at end of file
+null,0
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData6.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData6.result b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData6.result
index 58aaa20..4014f6d 100644
--- a/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData6.result
+++ b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData6.result
@@ -1,3 +1,3 @@
unique_key,maximum
-------------------------------
-0,0.0
\ No newline at end of file
+0,null
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData7.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData7.result b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData7.result
index e19a623..c2cc22e 100644
--- a/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData7.result
+++ b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData7.result
@@ -1,3 +1,3 @@
maximum,unique_key
-------------------------------
-0.0,0
\ No newline at end of file
+null,0
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData8.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData8.result b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData8.result
index e19a623..c2cc22e 100644
--- a/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData8.result
+++ b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupByWithNullData8.result
@@ -1,3 +1,3 @@
maximum,unique_key
-------------------------------
-0.0,0
\ No newline at end of file
+null,0
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/resources/results/TestJoinBroadcast/testLeftOuterJoinWithEmptyTable2.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestJoinBroadcast/testLeftOuterJoinWithEmptyTable2.result b/tajo-core/src/test/resources/results/TestJoinBroadcast/testLeftOuterJoinWithEmptyTable2.result
index 3bf0ed2..0830847 100644
--- a/tajo-core/src/test/resources/results/TestJoinBroadcast/testLeftOuterJoinWithEmptyTable2.result
+++ b/tajo-core/src/test/resources/results/TestJoinBroadcast/testLeftOuterJoinWithEmptyTable2.result
@@ -1,7 +1,7 @@
c_custkey,?sum,?max_1,?max_2
-------------------------------
-1,0,,
-2,0,,
-3,0,,
-4,0,,
-5,0,,
\ No newline at end of file
+1,null,null,null
+2,null,null,null
+3,null,null,null
+4,null,null,null
+5,null,null,null
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/resources/results/TestJoinBroadcast/testLeftOuterJoinWithEmptyTable4.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestJoinBroadcast/testLeftOuterJoinWithEmptyTable4.result b/tajo-core/src/test/resources/results/TestJoinBroadcast/testLeftOuterJoinWithEmptyTable4.result
index b9ac208..d4d0b53 100644
--- a/tajo-core/src/test/resources/results/TestJoinBroadcast/testLeftOuterJoinWithEmptyTable4.result
+++ b/tajo-core/src/test/resources/results/TestJoinBroadcast/testLeftOuterJoinWithEmptyTable4.result
@@ -1,4 +1,4 @@
?max,?sum_1,?max_2,?max_3
-------------------------------
5,6,O,1996-12-01
-5,0,,
\ No newline at end of file
+5,null,null,null
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable2.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable2.result b/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable2.result
index 3bf0ed2..0830847 100644
--- a/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable2.result
+++ b/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable2.result
@@ -1,7 +1,7 @@
c_custkey,?sum,?max_1,?max_2
-------------------------------
-1,0,,
-2,0,,
-3,0,,
-4,0,,
-5,0,,
\ No newline at end of file
+1,null,null,null
+2,null,null,null
+3,null,null,null
+4,null,null,null
+5,null,null,null
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable4.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable4.result b/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable4.result
index b9ac208..d4d0b53 100644
--- a/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable4.result
+++ b/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable4.result
@@ -1,4 +1,4 @@
?max,?sum_1,?max_2,?max_3
-------------------------------
5,6,O,1996-12-01
-5,0,,
\ No newline at end of file
+5,null,null,null
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/a1e03289/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable5.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable5.result b/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable5.result
index f1d80e4..61c58b9 100644
--- a/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable5.result
+++ b/tajo-core/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable5.result
@@ -1,4 +1,4 @@
l_linenumber,?sum,?max_1,?max_2,?avg_3,?sum_4
-------------------------------
-1,0,,,33.333333333333336,100.0
-2,0,,,42.5,85.0
\ No newline at end of file
+1,null,null,null,33.333333333333336,100.0
+2,null,null,null,42.5,85.0
\ No newline at end of file
[15/16] tajo git commit: Merge branch 'master' of
https://git-wip-us.apache.org/repos/asf/tajo into index_support
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/e04c65fd/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
index 0000000,bab5903..742665a
mode 000000,100644..100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
@@@ -1,0 -1,638 +1,650 @@@
+ /**
+ * 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.tajo.querymaster;
+
+ import org.apache.commons.logging.Log;
+ import org.apache.commons.logging.LogFactory;
+ import org.apache.hadoop.conf.Configuration;
+ import org.apache.hadoop.fs.FileStatus;
+ import org.apache.hadoop.fs.FileSystem;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.fs.permission.FsPermission;
+ import org.apache.hadoop.security.UserGroupInformation;
+ import org.apache.hadoop.service.CompositeService;
+ import org.apache.hadoop.util.StringUtils;
+ import org.apache.hadoop.yarn.event.AsyncDispatcher;
+ import org.apache.hadoop.yarn.event.EventHandler;
+ import org.apache.hadoop.yarn.util.Clock;
+ import org.apache.tajo.*;
+ import org.apache.tajo.algebra.Expr;
+ import org.apache.tajo.algebra.JsonHelper;
+ import org.apache.tajo.catalog.CatalogService;
+ import org.apache.tajo.catalog.TableDesc;
+ import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+ import org.apache.tajo.conf.TajoConf;
++import org.apache.tajo.plan.LogicalOptimizer;
++import org.apache.tajo.plan.LogicalPlan;
++import org.apache.tajo.plan.LogicalPlanner;
++import org.apache.tajo.plan.logical.LogicalRootNode;
++import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRule;
++import org.apache.tajo.plan.util.PlannerUtil;
+ import org.apache.tajo.engine.planner.global.MasterPlan;
+ import org.apache.tajo.engine.query.QueryContext;
+ import org.apache.tajo.exception.UnimplementedException;
+ import org.apache.tajo.ha.HAServiceUtil;
+ import org.apache.tajo.ipc.TajoMasterProtocol;
+ import org.apache.tajo.ipc.TajoWorkerProtocol;
+ import org.apache.tajo.master.TajoContainerProxy;
+ import org.apache.tajo.master.event.*;
+ import org.apache.tajo.master.rm.TajoWorkerResourceManager;
+ import org.apache.tajo.session.Session;
+ import org.apache.tajo.plan.LogicalOptimizer;
+ import org.apache.tajo.plan.LogicalPlan;
+ import org.apache.tajo.plan.LogicalPlanner;
+ import org.apache.tajo.plan.logical.LogicalNode;
-import org.apache.tajo.plan.logical.LogicalRootNode;
+ import org.apache.tajo.plan.logical.NodeType;
+ import org.apache.tajo.plan.logical.ScanNode;
-import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRule;
-import org.apache.tajo.plan.util.PlannerUtil;
+ import org.apache.tajo.plan.verifier.VerifyException;
+ import org.apache.tajo.rpc.NettyClientBase;
+ import org.apache.tajo.rpc.RpcConnectionPool;
+ import org.apache.tajo.storage.StorageManager;
+ import org.apache.tajo.storage.StorageProperty;
+ import org.apache.tajo.storage.StorageUtil;
+ import org.apache.tajo.util.metrics.TajoMetrics;
+ import org.apache.tajo.util.metrics.reporter.MetricsConsoleReporter;
+ import org.apache.tajo.worker.AbstractResourceAllocator;
+ import org.apache.tajo.worker.TajoResourceAllocator;
+
+ import java.io.IOException;
+ import java.util.*;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ import java.util.concurrent.atomic.AtomicLong;
+
+ import static org.apache.tajo.TajoProtos.QueryState;
+
+ public class QueryMasterTask extends CompositeService {
+ private static final Log LOG = LogFactory.getLog(QueryMasterTask.class.getName());
+
+ // query submission directory is private!
+ final public static FsPermission STAGING_DIR_PERMISSION =
+ FsPermission.createImmutable((short) 0700); // rwx--------
+
+ public static final String TMP_STAGING_DIR_PREFIX = ".staging";
+
+ private QueryId queryId;
+
+ private Session session;
+
+ private QueryContext queryContext;
+
+ private QueryMasterTaskContext queryTaskContext;
+
+ private QueryMaster.QueryMasterContext queryMasterContext;
+
+ private Query query;
+
+ private MasterPlan masterPlan;
+
+ private String jsonExpr;
+
+ private String logicalPlanJson;
+
+ private AsyncDispatcher dispatcher;
+
+ private final long querySubmitTime;
+
+ private Map<String, TableDesc> tableDescMap = new HashMap<String, TableDesc>();
+
+ private TajoConf systemConf;
+
+ private AtomicLong lastClientHeartbeat = new AtomicLong(-1);
+
+ private AbstractResourceAllocator resourceAllocator;
+
+ private AtomicBoolean stopped = new AtomicBoolean(false);
+
+ private TajoMetrics queryMetrics;
+
+ private Throwable initError;
+
+ private final List<TajoWorkerProtocol.TaskFatalErrorReport> diagnostics =
+ new ArrayList<TajoWorkerProtocol.TaskFatalErrorReport>();
+
+ public QueryMasterTask(QueryMaster.QueryMasterContext queryMasterContext,
+ QueryId queryId, Session session, QueryContext queryContext, String jsonExpr,
+ String logicalPlanJson) {
+
+ super(QueryMasterTask.class.getName());
+ this.queryMasterContext = queryMasterContext;
+ this.queryId = queryId;
+ this.session = session;
+ this.queryContext = queryContext;
+ this.jsonExpr = jsonExpr;
+ this.logicalPlanJson = logicalPlanJson;
+ this.querySubmitTime = System.currentTimeMillis();
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ systemConf = (TajoConf)conf;
+
+ try {
+ queryTaskContext = new QueryMasterTaskContext();
+ String resourceManagerClassName = systemConf.getVar(TajoConf.ConfVars.RESOURCE_MANAGER_CLASS);
+
+ if(resourceManagerClassName.indexOf(TajoWorkerResourceManager.class.getName()) >= 0) {
+ resourceAllocator = new TajoResourceAllocator(queryTaskContext);
+ } else {
+ throw new UnimplementedException(resourceManagerClassName + " is not supported yet");
+ }
+ addService(resourceAllocator);
+
+ dispatcher = new AsyncDispatcher();
+ addService(dispatcher);
+
+ dispatcher.register(StageEventType.class, new StageEventDispatcher());
+ dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
+ dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDispatcher());
+ dispatcher.register(QueryMasterQueryCompletedEvent.EventType.class, new QueryFinishEventHandler());
+ dispatcher.register(TaskSchedulerEvent.EventType.class, new TaskSchedulerDispatcher());
+ dispatcher.register(LocalTaskEventType.class, new LocalTaskEventHandler());
+
+ initStagingDir();
+
+ queryMetrics = new TajoMetrics(queryId.toString());
+
+ super.init(systemConf);
+ } catch (Throwable t) {
+ LOG.error(t.getMessage(), t);
+ initError = t;
+ }
+ }
+
+ public boolean isStopped() {
+ return stopped.get();
+ }
+
+ @Override
+ public void start() {
+ startQuery();
+ super.start();
+ }
+
+ @Override
+ public void stop() {
+
+ if(stopped.getAndSet(true)) {
+ return;
+ }
+
+ LOG.info("Stopping QueryMasterTask:" + queryId);
+
+ try {
+ resourceAllocator.stop();
+ } catch (Throwable t) {
+ LOG.fatal(t.getMessage(), t);
+ }
+
+ RpcConnectionPool connPool = RpcConnectionPool.getPool(queryMasterContext.getConf());
+ NettyClientBase tmClient = null;
+ try {
+ // In TajoMaster HA mode, if backup master be active status,
+ // worker may fail to connect existing active master. Thus,
+ // if worker can't connect the master, worker should try to connect another master and
+ // update master address in worker context.
+ if (systemConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
+ try {
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ } catch (Exception e) {
+ queryMasterContext.getWorkerContext().setWorkerResourceTrackerAddr(
+ HAServiceUtil.getResourceTrackerAddress(systemConf));
+ queryMasterContext.getWorkerContext().setTajoMasterAddress(
+ HAServiceUtil.getMasterUmbilicalAddress(systemConf));
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ }
+ } else {
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ }
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ } finally {
+ connPool.releaseConnection(tmClient);
+ }
+
+ super.stop();
+
+ //TODO change report to tajo master
+ if (queryMetrics != null) {
+ queryMetrics.report(new MetricsConsoleReporter());
+ }
+
+ LOG.info("Stopped QueryMasterTask:" + queryId);
+ }
+
+ public void handleTaskRequestEvent(TaskRequestEvent event) {
+ ExecutionBlockId id = event.getExecutionBlockId();
+ query.getStage(id).handleTaskRequestEvent(event);
+ }
+
+ public void handleTaskFailed(TajoWorkerProtocol.TaskFatalErrorReport report) {
+ synchronized(diagnostics) {
+ if (diagnostics.size() < 10) {
+ diagnostics.add(report);
+ }
+ }
+
+ getEventHandler().handle(new TaskFatalErrorEvent(report));
+ }
+
+ public Collection<TajoWorkerProtocol.TaskFatalErrorReport> getDiagnostics() {
+ synchronized(diagnostics) {
+ return Collections.unmodifiableCollection(diagnostics);
+ }
+ }
+
+ private class StageEventDispatcher implements EventHandler<StageEvent> {
+ public void handle(StageEvent event) {
+ ExecutionBlockId id = event.getStageId();
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("StageEventDispatcher:" + id + "," + event.getType());
+ }
+ query.getStage(id).handle(event);
+ }
+ }
+
+ private class TaskEventDispatcher
+ implements EventHandler<TaskEvent> {
+ public void handle(TaskEvent event) {
+ TaskId taskId = event.getTaskId();
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("TaskEventDispatcher>" + taskId + "," + event.getType());
+ }
+ Task task = query.getStage(taskId.getExecutionBlockId()).
+ getTask(taskId);
+ task.handle(event);
+ }
+ }
+
+ private class TaskAttemptEventDispatcher
+ implements EventHandler<TaskAttemptEvent> {
+ public void handle(TaskAttemptEvent event) {
+ TaskAttemptId attemptId = event.getTaskAttemptId();
+ Stage stage = query.getStage(attemptId.getTaskId().getExecutionBlockId());
+ Task task = stage.getTask(attemptId.getTaskId());
+ TaskAttempt attempt = task.getAttempt(attemptId);
+ attempt.handle(event);
+ }
+ }
+
+ private class TaskSchedulerDispatcher
+ implements EventHandler<TaskSchedulerEvent> {
+ public void handle(TaskSchedulerEvent event) {
+ Stage stage = query.getStage(event.getExecutionBlockId());
+ stage.getTaskScheduler().handle(event);
+ }
+ }
+
+ private class LocalTaskEventHandler implements EventHandler<LocalTaskEvent> {
+ @Override
+ public void handle(LocalTaskEvent event) {
+ TajoContainerProxy proxy = (TajoContainerProxy) resourceAllocator.getContainers().get(event.getContainerId());
+ if (proxy != null) {
+ proxy.killTaskAttempt(event.getTaskAttemptId());
+ }
+ }
+ }
+
+ private class QueryFinishEventHandler implements EventHandler<QueryMasterQueryCompletedEvent> {
+ @Override
+ public void handle(QueryMasterQueryCompletedEvent event) {
+ QueryId queryId = event.getQueryId();
+ LOG.info("Query completion notified from " + queryId);
+
+ while (!isTerminatedState(query.getSynchronizedState())) {
+ try {
+ synchronized (this) {
+ wait(10);
+ }
+ } catch (InterruptedException e) {
+ LOG.error(e);
+ }
+ }
+ LOG.info("Query final state: " + query.getSynchronizedState());
+
+ queryMasterContext.getEventHandler().handle(new QueryStopEvent(queryId));
+ }
+ }
+
+ private static boolean isTerminatedState(QueryState state) {
+ return
+ state == QueryState.QUERY_SUCCEEDED ||
+ state == QueryState.QUERY_FAILED ||
+ state == QueryState.QUERY_KILLED ||
+ state == QueryState.QUERY_ERROR;
+ }
+
+ public synchronized void startQuery() {
+ StorageManager sm = null;
+ LogicalPlan plan = null;
+ try {
+ if (query != null) {
+ LOG.warn("Query already started");
+ return;
+ }
++ LOG.info(SessionVars.INDEX_ENABLED.keyname() + " : " + queryContext.getBool(SessionVars.INDEX_ENABLED));
+ CatalogService catalog = getQueryTaskContext().getQueryMasterContext().getWorkerContext().getCatalog();
+ LogicalPlanner planner = new LogicalPlanner(catalog);
- LogicalOptimizer optimizer = new LogicalOptimizer(systemConf);
++ LogicalOptimizer optimizer = new LogicalOptimizer(systemConf, catalog);
+ Expr expr = JsonHelper.fromJson(jsonExpr, Expr.class);
+ jsonExpr = null; // remove the possible OOM
+ plan = planner.createPlan(queryContext, expr);
+
+ StoreType storeType = PlannerUtil.getStoreType(plan);
+ if (storeType != null) {
+ sm = StorageManager.getStorageManager(systemConf, storeType);
+ StorageProperty storageProperty = sm.getStorageProperty();
+ if (storageProperty.isSortedInsert()) {
+ String tableName = PlannerUtil.getStoreTableName(plan);
+ LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+ TableDesc tableDesc = PlannerUtil.getTableDesc(catalog, rootNode.getChild());
+ if (tableDesc == null) {
+ throw new VerifyException("Can't get table meta data from catalog: " + tableName);
+ }
+ List<LogicalPlanRewriteRule> storageSpecifiedRewriteRules = sm.getRewriteRules(
+ getQueryTaskContext().getQueryContext(), tableDesc);
+ if (storageSpecifiedRewriteRules != null) {
+ for (LogicalPlanRewriteRule eachRule: storageSpecifiedRewriteRules) {
+ optimizer.addRuleAfterToJoinOpt(eachRule);
+ }
+ }
+ }
+ }
+
+ optimizer.optimize(queryContext, plan);
+
+ for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
+ LogicalNode[] scanNodes = PlannerUtil.findAllNodes(block.getRoot(), NodeType.SCAN);
+ if (scanNodes != null) {
+ for (LogicalNode eachScanNode : scanNodes) {
+ ScanNode scanNode = (ScanNode) eachScanNode;
+ tableDescMap.put(scanNode.getCanonicalName(), scanNode.getTableDesc());
+ }
+ }
+
+ scanNodes = PlannerUtil.findAllNodes(block.getRoot(), NodeType.PARTITIONS_SCAN);
+ if (scanNodes != null) {
+ for (LogicalNode eachScanNode : scanNodes) {
+ ScanNode scanNode = (ScanNode) eachScanNode;
+ tableDescMap.put(scanNode.getCanonicalName(), scanNode.getTableDesc());
+ }
+ }
++
++ scanNodes = PlannerUtil.findAllNodes(block.getRoot(), NodeType.INDEX_SCAN);
++ if (scanNodes != null) {
++ for (LogicalNode eachScanNode : scanNodes) {
++ ScanNode scanNode = (ScanNode) eachScanNode;
++ tableDescMap.put(scanNode.getCanonicalName(), scanNode.getTableDesc());
++ }
++ }
+ }
+ MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
+ queryMasterContext.getGlobalPlanner().build(masterPlan);
+
+ query = new Query(queryTaskContext, queryId, querySubmitTime,
+ "", queryTaskContext.getEventHandler(), masterPlan);
+
+ dispatcher.register(QueryEventType.class, query);
+ queryTaskContext.getEventHandler().handle(new QueryEvent(queryId, QueryEventType.START));
+ } catch (Throwable t) {
+ LOG.error(t.getMessage(), t);
+ initError = t;
+
+ if (plan != null && sm != null) {
+ LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+ try {
+ sm.rollbackOutputCommit(rootNode.getChild());
+ } catch (IOException e) {
+ LOG.warn(query.getId() + ", failed processing cleanup storage when query failed:" + e.getMessage(), e);
+ }
+ }
+ }
+ }
+
+ private void initStagingDir() throws IOException {
+ Path stagingDir = null;
+ FileSystem defaultFS = TajoConf.getWarehouseDir(systemConf).getFileSystem(systemConf);
+
+ try {
+
+ stagingDir = initStagingDir(systemConf, queryId.toString(), queryContext);
+
+ // Create a subdirectories
+ LOG.info("The staging dir '" + stagingDir + "' is created.");
+ queryContext.setStagingDir(stagingDir);
+ } catch (IOException ioe) {
+ if (stagingDir != null && defaultFS.exists(stagingDir)) {
+ try {
+ defaultFS.delete(stagingDir, true);
+ LOG.info("The staging directory '" + stagingDir + "' is deleted");
+ } catch (Exception e) {
+ LOG.warn(e.getMessage());
+ }
+ }
+
+ throw ioe;
+ }
+ }
+
+ /**
+ * It initializes the final output and staging directory and sets
+ * them to variables.
+ */
+ public static Path initStagingDir(TajoConf conf, String queryId, QueryContext context) throws IOException {
+
+ String realUser;
+ String currentUser;
+ UserGroupInformation ugi;
+ ugi = UserGroupInformation.getLoginUser();
+ realUser = ugi.getShortUserName();
+ currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
+
+ FileSystem fs;
+ Path stagingDir;
+
+ ////////////////////////////////////////////
+ // Create Output Directory
+ ////////////////////////////////////////////
+
+ String outputPath = context.get(QueryVars.OUTPUT_TABLE_PATH, "");
+ if (context.isCreateTable() || context.isInsert()) {
+ if (outputPath == null || outputPath.isEmpty()) {
+ // hbase
+ stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
+ } else {
+ stagingDir = StorageUtil.concatPath(context.getOutputPath(), TMP_STAGING_DIR_PREFIX, queryId);
+ }
+ } else {
+ stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
+ }
+
+ // initializ
+ fs = stagingDir.getFileSystem(conf);
+
+ if (fs.exists(stagingDir)) {
+ throw new IOException("The staging directory '" + stagingDir + "' already exists");
+ }
+ fs.mkdirs(stagingDir, new FsPermission(STAGING_DIR_PERMISSION));
+ FileStatus fsStatus = fs.getFileStatus(stagingDir);
+ String owner = fsStatus.getOwner();
+
+ if (!owner.isEmpty() && !(owner.equals(currentUser) || owner.equals(realUser))) {
+ throw new IOException("The ownership on the user's query " +
+ "directory " + stagingDir + " is not as expected. " +
+ "It is owned by " + owner + ". The directory must " +
+ "be owned by the submitter " + currentUser + " or " +
+ "by " + realUser);
+ }
+
+ if (!fsStatus.getPermission().equals(STAGING_DIR_PERMISSION)) {
+ LOG.info("Permissions on staging directory " + stagingDir + " are " +
+ "incorrect: " + fsStatus.getPermission() + ". Fixing permissions " +
+ "to correct value " + STAGING_DIR_PERMISSION);
+ fs.setPermission(stagingDir, new FsPermission(STAGING_DIR_PERMISSION));
+ }
+
+ Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+ fs.mkdirs(stagingResultDir);
+
+ return stagingDir;
+ }
+
+ public Query getQuery() {
+ return query;
+ }
+
+ protected void expireQuerySession() {
+ if(!isTerminatedState(query.getState()) && !(query.getState() == QueryState.QUERY_KILL_WAIT)){
+ query.handle(new QueryEvent(queryId, QueryEventType.KILL));
+ }
+ }
+
+ public QueryMasterTaskContext getQueryTaskContext() {
+ return queryTaskContext;
+ }
+
+ public EventHandler getEventHandler() {
+ return queryTaskContext.getEventHandler();
+ }
+
+ public void touchSessionTime() {
+ this.lastClientHeartbeat.set(System.currentTimeMillis());
+ }
+
+ public long getLastClientHeartbeat() {
+ return this.lastClientHeartbeat.get();
+ }
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public boolean isInitError() {
+ return initError != null;
+ }
+
+ public QueryState getState() {
+ if(query == null) {
+ if (isInitError()) {
+ return QueryState.QUERY_ERROR;
+ } else {
+ return QueryState.QUERY_NOT_ASSIGNED;
+ }
+ } else {
+ return query.getState();
+ }
+ }
+
+ public Throwable getInitError() {
+ return initError;
+ }
+
+ public String getErrorMessage() {
+ if (isInitError()) {
+ return StringUtils.stringifyException(initError);
+ } else {
+ return null;
+ }
+ }
+
+ public long getQuerySubmitTime() {
+ return this.querySubmitTime;
+ }
+
+ public class QueryMasterTaskContext {
+ EventHandler eventHandler;
+ public QueryMaster.QueryMasterContext getQueryMasterContext() {
+ return queryMasterContext;
+ }
+
+ public Session getSession() {
+ return session;
+ }
+
+ public QueryContext getQueryContext() {
+ return queryContext;
+ }
+
+ public TajoConf getConf() {
+ return systemConf;
+ }
+
+ public Clock getClock() {
+ return queryMasterContext.getClock();
+ }
+
+ public Query getQuery() {
+ return query;
+ }
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public Path getStagingDir() {
+ return queryContext.getStagingDir();
+ }
+
+ public synchronized EventHandler getEventHandler() {
+ if(eventHandler == null) {
+ eventHandler = dispatcher.getEventHandler();
+ }
+ return eventHandler;
+ }
+
+ public AsyncDispatcher getDispatcher() {
+ return dispatcher;
+ }
+
+ public Stage getStage(ExecutionBlockId id) {
+ return query.getStage(id);
+ }
+
+ public Map<String, TableDesc> getTableDescMap() {
+ return tableDescMap;
+ }
+
+ public float getProgress() {
+ if(query == null) {
+ return 0.0f;
+ }
+ return query.getProgress();
+ }
+
+ public AbstractResourceAllocator getResourceAllocator() {
+ return resourceAllocator;
+ }
+
+ public TajoMetrics getQueryMetrics() {
+ return queryMetrics;
+ }
+ }
+ }
http://git-wip-us.apache.org/repos/asf/tajo/blob/e04c65fd/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
index 6a13898,2ae4bed..c10d3b7
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
@@@ -31,10 -29,9 +31,10 @@@ import org.apache.tajo.annotation.Nulla
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.ipc.ClientProtos.GetQueryHistoryResponse;
import org.apache.tajo.ipc.ClientProtos.QueryIdRequest;
+import org.apache.tajo.ipc.ClientProtos.RequestResult;
import org.apache.tajo.ipc.ClientProtos.ResultCode;
import org.apache.tajo.ipc.QueryMasterClientProtocol;
- import org.apache.tajo.master.querymaster.QueryMasterTask;
+ import org.apache.tajo.querymaster.QueryMasterTask;
import org.apache.tajo.rpc.BlockingRpcServer;
import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
import org.apache.tajo.util.NetUtils;
http://git-wip-us.apache.org/repos/asf/tajo/blob/e04c65fd/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/e04c65fd/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/e04c65fd/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/e04c65fd/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
----------------------------------------------------------------------
diff --cc tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
index 0000000,a125196..4526863
mode 000000,100644..100644
--- a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
@@@ -1,0 -1,125 +1,125 @@@
+ /**
+ * 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.tajo.querymaster;
+
+ import org.apache.tajo.*;
+ import org.apache.tajo.algebra.Expr;
+ import org.apache.tajo.benchmark.TPCH;
+ import org.apache.tajo.catalog.CatalogService;
+ import org.apache.tajo.client.TajoClient;
+ import org.apache.tajo.client.TajoClientImpl;
+ import org.apache.tajo.conf.TajoConf;
+ import org.apache.tajo.engine.parser.SQLAnalyzer;
+ import org.apache.tajo.engine.planner.global.GlobalPlanner;
+ import org.apache.tajo.engine.planner.global.MasterPlan;
+ import org.apache.tajo.engine.query.QueryContext;
+ import org.apache.tajo.master.event.QueryEvent;
+ import org.apache.tajo.master.event.QueryEventType;
+ import org.apache.tajo.session.Session;
+ import org.apache.tajo.plan.LogicalOptimizer;
+ import org.apache.tajo.plan.LogicalPlan;
+ import org.apache.tajo.plan.LogicalPlanner;
+ import org.junit.AfterClass;
+ import org.junit.BeforeClass;
+ import org.junit.Test;
+
+ import java.io.File;
+ import java.io.IOException;
+
+ import static org.junit.Assert.*;
+
+ public class TestKillQuery {
+ private static TajoTestingCluster cluster;
+ private static TajoConf conf;
+ private static TajoClient client;
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ cluster = new TajoTestingCluster();
+ cluster.startMiniClusterInLocal(1);
+ conf = cluster.getConfiguration();
+ client = new TajoClientImpl(cluster.getConfiguration());
+ File file = TPCH.getDataFile("lineitem");
+ client.executeQueryAndGetResult("create external table default.lineitem (l_orderkey int, l_partkey int) "
+ + "using text location 'file://" + file.getAbsolutePath() + "'");
+ assertTrue(client.existTable("default.lineitem"));
+ }
+
+ @AfterClass
+ public static void tearDown() throws IOException {
+ if (client != null) client.close();
+ if (cluster != null) cluster.shutdownMiniCluster();
+ }
+
+ @Test
+ public final void testKillQueryFromInitState() throws Exception {
+ SQLAnalyzer analyzer = new SQLAnalyzer();
+ QueryContext defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
+ Session session = LocalTajoTestingUtility.createDummySession();
+ CatalogService catalog = cluster.getMaster().getCatalog();
+ String query = "select l_orderkey, l_partkey from lineitem group by l_orderkey, l_partkey order by l_orderkey";
+
+ LogicalPlanner planner = new LogicalPlanner(catalog);
- LogicalOptimizer optimizer = new LogicalOptimizer(conf);
++ LogicalOptimizer optimizer = new LogicalOptimizer(conf, catalog);
+ Expr expr = analyzer.parse(query);
+ LogicalPlan plan = planner.createPlan(defaultContext, expr);
+
+ optimizer.optimize(plan);
+
+ QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
+ QueryContext queryContext = new QueryContext(conf);
+ MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
+ GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
+ globalPlanner.build(masterPlan);
+
+ QueryMaster qm = cluster.getTajoWorkers().get(0).getWorkerContext().getQueryMaster();
+ QueryMasterTask queryMasterTask = new QueryMasterTask(qm.getContext(),
+ queryId, session, defaultContext, expr.toJson(), plan.getRootBlock().getRoot().toJson());
+
+ queryMasterTask.init(conf);
+ queryMasterTask.getQueryTaskContext().getDispatcher().start();
+ queryMasterTask.startQuery();
+
+ try{
+ cluster.waitForQueryState(queryMasterTask.getQuery(), TajoProtos.QueryState.QUERY_RUNNING, 2);
+ } finally {
+ assertEquals(TajoProtos.QueryState.QUERY_RUNNING, queryMasterTask.getQuery().getSynchronizedState());
+ }
+
+ Stage stage = queryMasterTask.getQuery().getStages().iterator().next();
+ assertNotNull(stage);
+
+ try{
+ cluster.waitForStageState(stage, StageState.INITED, 2);
+ } finally {
+ assertEquals(StageState.INITED, stage.getSynchronizedState());
+ }
+
+ // fire kill event
+ Query q = queryMasterTask.getQuery();
+ q.handle(new QueryEvent(queryId, QueryEventType.KILL));
+
+ try{
+ cluster.waitForQueryState(queryMasterTask.getQuery(), TajoProtos.QueryState.QUERY_KILLED, 50);
+ } finally {
+ assertEquals(TajoProtos.QueryState.QUERY_KILLED, queryMasterTask.getQuery().getSynchronizedState());
+ }
+ queryMasterTask.stop();
+ }
+ }
[06/16] tajo git commit: TAJO-1288: Refactoring
org.apache.tajo.master package.
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java
new file mode 100644
index 0000000..76df397
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMaster.java
@@ -0,0 +1,631 @@
+/**
+ * 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.tajo.querymaster;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tajo.*;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.global.GlobalPlanner;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.ha.HAServiceUtil;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.event.QueryStartEvent;
+import org.apache.tajo.master.event.QueryStopEvent;
+import org.apache.tajo.rpc.CallFuture;
+import org.apache.tajo.rpc.NettyClientBase;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.rpc.RpcConnectionPool;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.util.history.QueryHistory;
+import org.apache.tajo.worker.TajoWorker;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.tajo.ipc.TajoMasterProtocol.TajoHeartbeat;
+import static org.apache.tajo.ipc.TajoMasterProtocol.TajoHeartbeatResponse;
+
+// TODO - when exception, send error status to QueryJobManager
+public class QueryMaster extends CompositeService implements EventHandler {
+ private static final Log LOG = LogFactory.getLog(QueryMaster.class.getName());
+
+ private int querySessionTimeout;
+
+ private Clock clock;
+
+ private AsyncDispatcher dispatcher;
+
+ private GlobalPlanner globalPlanner;
+
+ private TajoConf systemConf;
+
+ private Map<QueryId, QueryMasterTask> queryMasterTasks = Maps.newConcurrentMap();
+
+ private Map<QueryId, QueryMasterTask> finishedQueryMasterTasks = Maps.newConcurrentMap();
+
+ private ClientSessionTimeoutCheckThread clientSessionTimeoutCheckThread;
+
+ private AtomicBoolean queryMasterStop = new AtomicBoolean(false);
+
+ private QueryMasterContext queryMasterContext;
+
+ private QueryContext queryContext;
+
+ private QueryHeartbeatThread queryHeartbeatThread;
+
+ private FinishedQueryMasterTaskCleanThread finishedQueryMasterTaskCleanThread;
+
+ private TajoWorker.WorkerContext workerContext;
+
+ private RpcConnectionPool connPool;
+
+ private ExecutorService eventExecutor;
+
+ public QueryMaster(TajoWorker.WorkerContext workerContext) {
+ super(QueryMaster.class.getName());
+ this.workerContext = workerContext;
+ }
+
+ public void init(Configuration conf) {
+ LOG.info("QueryMaster init");
+ try {
+ this.systemConf = (TajoConf)conf;
+ this.connPool = RpcConnectionPool.getPool(systemConf);
+
+ querySessionTimeout = systemConf.getIntVar(TajoConf.ConfVars.QUERY_SESSION_TIMEOUT);
+ queryMasterContext = new QueryMasterContext(systemConf);
+
+ clock = new SystemClock();
+
+ this.dispatcher = new AsyncDispatcher();
+ addIfService(dispatcher);
+
+ globalPlanner = new GlobalPlanner(systemConf, workerContext);
+
+ dispatcher.register(QueryStartEvent.EventType.class, new QueryStartEventHandler());
+ dispatcher.register(QueryStopEvent.EventType.class, new QueryStopEventHandler());
+
+ } catch (Throwable t) {
+ LOG.error(t.getMessage(), t);
+ throw new RuntimeException(t);
+ }
+ super.init(conf);
+ }
+
+ @Override
+ public void start() {
+ LOG.info("QueryMaster start");
+
+ queryHeartbeatThread = new QueryHeartbeatThread();
+ queryHeartbeatThread.start();
+
+ clientSessionTimeoutCheckThread = new ClientSessionTimeoutCheckThread();
+ clientSessionTimeoutCheckThread.start();
+
+ finishedQueryMasterTaskCleanThread = new FinishedQueryMasterTaskCleanThread();
+ finishedQueryMasterTaskCleanThread.start();
+
+ eventExecutor = Executors.newSingleThreadExecutor();
+ super.start();
+ }
+
+ @Override
+ public void stop() {
+ if(queryMasterStop.getAndSet(true)){
+ return;
+ }
+
+ if(queryHeartbeatThread != null) {
+ queryHeartbeatThread.interrupt();
+ }
+
+ if(clientSessionTimeoutCheckThread != null) {
+ clientSessionTimeoutCheckThread.interrupt();
+ }
+
+ if(finishedQueryMasterTaskCleanThread != null) {
+ finishedQueryMasterTaskCleanThread.interrupt();
+ }
+
+ if(eventExecutor != null){
+ eventExecutor.shutdown();
+ }
+
+ super.stop();
+
+ LOG.info("QueryMaster stop");
+ if(queryMasterContext.getWorkerContext().isYarnContainerMode()) {
+ queryMasterContext.getWorkerContext().stopWorker(true);
+ }
+ }
+
+ protected void cleanupExecutionBlock(List<TajoIdProtos.ExecutionBlockIdProto> executionBlockIds) {
+ StringBuilder cleanupMessage = new StringBuilder();
+ String prefix = "";
+ for (TajoIdProtos.ExecutionBlockIdProto eachEbId: executionBlockIds) {
+ cleanupMessage.append(prefix).append(new ExecutionBlockId(eachEbId).toString());
+ prefix = ",";
+ }
+ LOG.info("cleanup executionBlocks: " + cleanupMessage);
+ NettyClientBase rpc = null;
+ List<TajoMasterProtocol.WorkerResourceProto> workers = getAllWorker();
+ TajoWorkerProtocol.ExecutionBlockListProto.Builder builder = TajoWorkerProtocol.ExecutionBlockListProto.newBuilder();
+ builder.addAllExecutionBlockId(Lists.newArrayList(executionBlockIds));
+ TajoWorkerProtocol.ExecutionBlockListProto executionBlockListProto = builder.build();
+
+ for (TajoMasterProtocol.WorkerResourceProto worker : workers) {
+ try {
+ TajoProtos.WorkerConnectionInfoProto connectionInfo = worker.getConnectionInfo();
+ rpc = connPool.getConnection(NetUtils.createSocketAddr(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()),
+ TajoWorkerProtocol.class, true);
+ TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerProtocolService = rpc.getStub();
+
+ tajoWorkerProtocolService.cleanupExecutionBlocks(null, executionBlockListProto, NullCallback.get());
+ } catch (Exception e) {
+ continue;
+ } finally {
+ connPool.releaseConnection(rpc);
+ }
+ }
+ }
+
+ private void cleanup(QueryId queryId) {
+ LOG.info("cleanup query resources : " + queryId);
+ NettyClientBase rpc = null;
+ List<TajoMasterProtocol.WorkerResourceProto> workers = getAllWorker();
+
+ for (TajoMasterProtocol.WorkerResourceProto worker : workers) {
+ try {
+ TajoProtos.WorkerConnectionInfoProto connectionInfo = worker.getConnectionInfo();
+ rpc = connPool.getConnection(NetUtils.createSocketAddr(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()),
+ TajoWorkerProtocol.class, true);
+ TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerProtocolService = rpc.getStub();
+
+ tajoWorkerProtocolService.cleanup(null, queryId.getProto(), NullCallback.get());
+ } catch (Exception e) {
+ LOG.error(e.getMessage());
+ } finally {
+ connPool.releaseConnection(rpc);
+ }
+ }
+ }
+
+ public List<TajoMasterProtocol.WorkerResourceProto> getAllWorker() {
+
+ NettyClientBase rpc = null;
+ try {
+ // In TajoMaster HA mode, if backup master be active status,
+ // worker may fail to connect existing active master. Thus,
+ // if worker can't connect the master, worker should try to connect another master and
+ // update master address in worker context.
+ if (systemConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
+ try {
+ rpc = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ } catch (Exception e) {
+ queryMasterContext.getWorkerContext().setWorkerResourceTrackerAddr(
+ HAServiceUtil.getResourceTrackerAddress(systemConf));
+ queryMasterContext.getWorkerContext().setTajoMasterAddress(
+ HAServiceUtil.getMasterUmbilicalAddress(systemConf));
+ rpc = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ }
+ } else {
+ rpc = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ }
+
+ TajoMasterProtocol.TajoMasterProtocolService masterService = rpc.getStub();
+
+ CallFuture<TajoMasterProtocol.WorkerResourcesRequest> callBack =
+ new CallFuture<TajoMasterProtocol.WorkerResourcesRequest>();
+ masterService.getAllWorkerResource(callBack.getController(),
+ PrimitiveProtos.NullProto.getDefaultInstance(), callBack);
+
+ TajoMasterProtocol.WorkerResourcesRequest workerResourcesRequest = callBack.get(2, TimeUnit.SECONDS);
+ return workerResourcesRequest.getWorkerResourcesList();
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ } finally {
+ connPool.releaseConnection(rpc);
+ }
+ return new ArrayList<TajoMasterProtocol.WorkerResourceProto>();
+ }
+
+ public void reportQueryStatusToQueryMaster(QueryId queryId, TajoProtos.QueryState state) {
+ LOG.info("Send QueryMaster Ready to QueryJobManager:" + queryId);
+ NettyClientBase tmClient = null;
+ try {
+ // In TajoMaster HA mode, if backup master be active status,
+ // worker may fail to connect existing active master. Thus,
+ // if worker can't connect the master, worker should try to connect another master and
+ // update master address in worker context.
+ if (systemConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
+ try {
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ } catch (Exception e) {
+ queryMasterContext.getWorkerContext().setWorkerResourceTrackerAddr(
+ HAServiceUtil.getResourceTrackerAddress(systemConf));
+ queryMasterContext.getWorkerContext().setTajoMasterAddress(
+ HAServiceUtil.getMasterUmbilicalAddress(systemConf));
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ }
+ } else {
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ }
+
+ TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
+
+ TajoHeartbeat.Builder queryHeartbeatBuilder = TajoHeartbeat.newBuilder()
+ .setConnectionInfo(workerContext.getConnectionInfo().getProto())
+ .setState(state)
+ .setQueryId(queryId.getProto());
+
+ CallFuture<TajoHeartbeatResponse> callBack =
+ new CallFuture<TajoHeartbeatResponse>();
+
+ masterClientService.heartbeat(callBack.getController(), queryHeartbeatBuilder.build(), callBack);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ } finally {
+ connPool.releaseConnection(tmClient);
+ }
+ }
+
+ @Override
+ public void handle(Event event) {
+ dispatcher.getEventHandler().handle(event);
+ }
+
+ public Query getQuery(QueryId queryId) {
+ return queryMasterTasks.get(queryId).getQuery();
+ }
+
+ public QueryMasterTask getQueryMasterTask(QueryId queryId) {
+ return queryMasterTasks.get(queryId);
+ }
+
+ public QueryMasterTask getQueryMasterTask(QueryId queryId, boolean includeFinished) {
+ QueryMasterTask queryMasterTask = queryMasterTasks.get(queryId);
+ if(queryMasterTask != null) {
+ return queryMasterTask;
+ } else {
+ if(includeFinished) {
+ return finishedQueryMasterTasks.get(queryId);
+ } else {
+ return null;
+ }
+ }
+ }
+
+ public QueryMasterContext getContext() {
+ return this.queryMasterContext;
+ }
+
+ public Collection<QueryMasterTask> getQueryMasterTasks() {
+ return queryMasterTasks.values();
+ }
+
+ public Collection<QueryMasterTask> getFinishedQueryMasterTasks() {
+ return finishedQueryMasterTasks.values();
+ }
+
+ public class QueryMasterContext {
+ private TajoConf conf;
+
+ public QueryMasterContext(TajoConf conf) {
+ this.conf = conf;
+ }
+
+ public TajoConf getConf() {
+ return conf;
+ }
+
+ public ExecutorService getEventExecutor(){
+ return eventExecutor;
+ }
+
+ public AsyncDispatcher getDispatcher() {
+ return dispatcher;
+ }
+
+ public Clock getClock() {
+ return clock;
+ }
+
+ public QueryMaster getQueryMaster() {
+ return QueryMaster.this;
+ }
+
+ public GlobalPlanner getGlobalPlanner() {
+ return globalPlanner;
+ }
+
+ public TajoWorker.WorkerContext getWorkerContext() {
+ return workerContext;
+ }
+
+ public EventHandler getEventHandler() {
+ return dispatcher.getEventHandler();
+ }
+
+ public void stopQuery(QueryId queryId) {
+ QueryMasterTask queryMasterTask = queryMasterTasks.remove(queryId);
+ if(queryMasterTask == null) {
+ LOG.warn("No query info:" + queryId);
+ return;
+ }
+
+ finishedQueryMasterTasks.put(queryId, queryMasterTask);
+
+ TajoHeartbeat queryHeartbeat = buildTajoHeartBeat(queryMasterTask);
+ CallFuture<TajoHeartbeatResponse> future = new CallFuture<TajoHeartbeatResponse>();
+
+ NettyClientBase tmClient = null;
+ try {
+ // In TajoMaster HA mode, if backup master be active status,
+ // worker may fail to connect existing active master. Thus,
+ // if worker can't connect the master, worker should try to connect another master and
+ // update master address in worker context.
+ if (systemConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
+ try {
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ } catch (Exception e) {
+ queryMasterContext.getWorkerContext().setWorkerResourceTrackerAddr(HAServiceUtil.getResourceTrackerAddress(systemConf));
+ queryMasterContext.getWorkerContext().setTajoMasterAddress(HAServiceUtil.getMasterUmbilicalAddress(systemConf));
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ }
+ } else {
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ }
+
+ TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
+ masterClientService.heartbeat(future.getController(), queryHeartbeat, future);
+ } catch (Exception e) {
+ //this function will be closed in new thread.
+ //When tajo do stop cluster, tajo master maybe throw closed connection exception
+
+ LOG.error(e.getMessage(), e);
+ } finally {
+ connPool.releaseConnection(tmClient);
+ }
+
+ try {
+ queryMasterTask.stop();
+ if (!queryContext.getBool(SessionVars.DEBUG_ENABLED)) {
+ cleanup(queryId);
+ }
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ Query query = queryMasterTask.getQuery();
+ if (query != null) {
+ QueryHistory queryHisory = query.getQueryHistory();
+ if (queryHisory != null) {
+ query.context.getQueryMasterContext().getWorkerContext().
+ getTaskHistoryWriter().appendHistory(queryHisory);
+ }
+ }
+ if(workerContext.isYarnContainerMode()) {
+ stop();
+ }
+ }
+ }
+
+ private TajoHeartbeat buildTajoHeartBeat(QueryMasterTask queryMasterTask) {
+ TajoHeartbeat.Builder builder = TajoHeartbeat.newBuilder();
+
+ builder.setConnectionInfo(workerContext.getConnectionInfo().getProto());
+ builder.setQueryId(queryMasterTask.getQueryId().getProto());
+ builder.setState(queryMasterTask.getState());
+ if (queryMasterTask.getQuery() != null) {
+ if (queryMasterTask.getQuery().getResultDesc() != null) {
+ builder.setResultDesc(queryMasterTask.getQuery().getResultDesc().getProto());
+ }
+ builder.setQueryProgress(queryMasterTask.getQuery().getProgress());
+ builder.setQueryFinishTime(queryMasterTask.getQuery().getFinishTime());
+ }
+ return builder.build();
+ }
+
+ private class QueryStartEventHandler implements EventHandler<QueryStartEvent> {
+ @Override
+ public void handle(QueryStartEvent event) {
+ LOG.info("Start QueryStartEventHandler:" + event.getQueryId());
+ QueryMasterTask queryMasterTask = new QueryMasterTask(queryMasterContext,
+ event.getQueryId(), event.getSession(), event.getQueryContext(), event.getJsonExpr(), event.getLogicalPlanJson());
+
+ synchronized(queryMasterTasks) {
+ queryMasterTasks.put(event.getQueryId(), queryMasterTask);
+ }
+
+ queryMasterTask.init(systemConf);
+ if (!queryMasterTask.isInitError()) {
+ queryMasterTask.start();
+ }
+
+ queryContext = event.getQueryContext();
+
+ if (queryMasterTask.isInitError()) {
+ queryMasterContext.stopQuery(queryMasterTask.getQueryId());
+ return;
+ }
+ }
+ }
+
+ private class QueryStopEventHandler implements EventHandler<QueryStopEvent> {
+ @Override
+ public void handle(QueryStopEvent event) {
+ queryMasterContext.stopQuery(event.getQueryId());
+ }
+ }
+
+ class QueryHeartbeatThread extends Thread {
+ public QueryHeartbeatThread() {
+ super("QueryHeartbeatThread");
+ }
+
+ @Override
+ public void run() {
+ LOG.info("Start QueryMaster heartbeat thread");
+ while(!queryMasterStop.get()) {
+ List<QueryMasterTask> tempTasks = new ArrayList<QueryMasterTask>();
+ synchronized(queryMasterTasks) {
+ tempTasks.addAll(queryMasterTasks.values());
+ }
+ synchronized(queryMasterTasks) {
+ for(QueryMasterTask eachTask: tempTasks) {
+ NettyClientBase tmClient;
+ try {
+ // In TajoMaster HA mode, if backup master be active status,
+ // worker may fail to connect existing active master. Thus,
+ // if worker can't connect the master, worker should try to connect another master and
+ // update master address in worker context.
+ if (systemConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
+ try {
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ } catch (Exception e) {
+ queryMasterContext.getWorkerContext().setWorkerResourceTrackerAddr(
+ HAServiceUtil.getResourceTrackerAddress(systemConf));
+ queryMasterContext.getWorkerContext().setTajoMasterAddress(
+ HAServiceUtil.getMasterUmbilicalAddress(systemConf));
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ }
+ } else {
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ }
+
+ TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
+
+ CallFuture<TajoHeartbeatResponse> callBack =
+ new CallFuture<TajoHeartbeatResponse>();
+
+ TajoHeartbeat queryHeartbeat = buildTajoHeartBeat(eachTask);
+ masterClientService.heartbeat(callBack.getController(), queryHeartbeat, callBack);
+ } catch (Throwable t) {
+ t.printStackTrace();
+ }
+ }
+ }
+ synchronized(queryMasterStop) {
+ try {
+ queryMasterStop.wait(2000);
+ } catch (InterruptedException e) {
+ break;
+ }
+ }
+ }
+ LOG.info("QueryMaster heartbeat thread stopped");
+ }
+ }
+
+ class ClientSessionTimeoutCheckThread extends Thread {
+ public void run() {
+ LOG.info("ClientSessionTimeoutCheckThread started");
+ while(!queryMasterStop.get()) {
+ try {
+ Thread.sleep(1000);
+ } catch (InterruptedException e) {
+ break;
+ }
+ List<QueryMasterTask> tempTasks = new ArrayList<QueryMasterTask>();
+ synchronized(queryMasterTasks) {
+ tempTasks.addAll(queryMasterTasks.values());
+ }
+
+ for(QueryMasterTask eachTask: tempTasks) {
+ if(!eachTask.isStopped()) {
+ try {
+ long lastHeartbeat = eachTask.getLastClientHeartbeat();
+ long time = System.currentTimeMillis() - lastHeartbeat;
+ if(lastHeartbeat > 0 && time > querySessionTimeout * 1000) {
+ LOG.warn("Query " + eachTask.getQueryId() + " stopped cause query session timeout: " + time + " ms");
+ eachTask.expireQuerySession();
+ }
+ } catch (Exception e) {
+ LOG.error(eachTask.getQueryId() + ":" + e.getMessage(), e);
+ }
+ }
+ }
+ }
+ }
+ }
+
+ class FinishedQueryMasterTaskCleanThread extends Thread {
+ public void run() {
+ int expireIntervalTime = systemConf.getIntVar(TajoConf.ConfVars.WORKER_HISTORY_EXPIRE_PERIOD);
+ LOG.info("FinishedQueryMasterTaskCleanThread started: expire interval minutes = " + expireIntervalTime);
+ while(!queryMasterStop.get()) {
+ try {
+ Thread.sleep(60 * 1000 * 60); // hourly
+ } catch (InterruptedException e) {
+ break;
+ }
+ try {
+ long expireTime = System.currentTimeMillis() - expireIntervalTime * 60 * 1000;
+ cleanExpiredFinishedQueryMasterTask(expireTime);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ }
+ }
+
+ private void cleanExpiredFinishedQueryMasterTask(long expireTime) {
+ synchronized(finishedQueryMasterTasks) {
+ List<QueryId> expiredQueryIds = new ArrayList<QueryId>();
+ for(Map.Entry<QueryId, QueryMasterTask> entry: finishedQueryMasterTasks.entrySet()) {
+ if(entry.getValue().getStartTime() < expireTime) {
+ expiredQueryIds.add(entry.getKey());
+ }
+ }
+
+ for(QueryId eachId: expiredQueryIds) {
+ finishedQueryMasterTasks.remove(eachId);
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java
new file mode 100644
index 0000000..4a91326
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterManagerService.java
@@ -0,0 +1,262 @@
+/**
+ * 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.tajo.querymaster;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.tajo.*;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.ipc.QueryMasterProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.container.TajoContainerId;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.session.Session;
+import org.apache.tajo.rpc.AsyncRpcServer;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.worker.TajoWorker;
+
+import java.net.InetSocketAddress;
+
+public class QueryMasterManagerService extends CompositeService
+ implements QueryMasterProtocol.QueryMasterProtocolService.Interface {
+ private static final Log LOG = LogFactory.getLog(QueryMasterManagerService.class.getName());
+
+ private AsyncRpcServer rpcServer;
+ private InetSocketAddress bindAddr;
+ private String addr;
+ private int port;
+
+ private QueryMaster queryMaster;
+
+ private TajoWorker.WorkerContext workerContext;
+
+ public QueryMasterManagerService(TajoWorker.WorkerContext workerContext, int port) {
+ super(QueryMasterManagerService.class.getName());
+ this.workerContext = workerContext;
+ this.port = port;
+ }
+
+ public QueryMaster getQueryMaster() {
+ return queryMaster;
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ Preconditions.checkArgument(conf instanceof TajoConf);
+ TajoConf tajoConf = (TajoConf) conf;
+ try {
+ // Setup RPC server
+ InetSocketAddress initIsa =
+ new InetSocketAddress("0.0.0.0", port);
+ if (initIsa.getAddress() == null) {
+ throw new IllegalArgumentException("Failed resolve of " + initIsa);
+ }
+
+ int workerNum = tajoConf.getIntVar(TajoConf.ConfVars.QUERY_MASTER_RPC_SERVER_WORKER_THREAD_NUM);
+ this.rpcServer = new AsyncRpcServer(QueryMasterProtocol.class, this, initIsa, workerNum);
+ this.rpcServer.start();
+
+ this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress());
+ this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort();
+
+ this.port = bindAddr.getPort();
+
+ queryMaster = new QueryMaster(workerContext);
+ addService(queryMaster);
+
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ // Get the master address
+ LOG.info("QueryMasterManagerService is bind to " + addr);
+ ((TajoConf)conf).setVar(TajoConf.ConfVars.WORKER_QM_RPC_ADDRESS, addr);
+
+ super.init(conf);
+ }
+
+ @Override
+ public void start() {
+ super.start();
+ }
+
+ @Override
+ public void stop() {
+ if(rpcServer != null) {
+ rpcServer.shutdown();
+ }
+ LOG.info("QueryMasterManagerService stopped");
+ super.stop();
+ }
+
+ public InetSocketAddress getBindAddr() {
+ return bindAddr;
+ }
+
+ public String getHostAndPort() {
+ return bindAddr.getHostName() + ":" + bindAddr.getPort();
+ }
+
+ @Override
+ public void getTask(RpcController controller, TajoWorkerProtocol.GetTaskRequestProto request,
+ RpcCallback<TajoWorkerProtocol.TaskRequestProto> done) {
+ try {
+ ExecutionBlockId ebId = new ExecutionBlockId(request.getExecutionBlockId());
+ QueryMasterTask queryMasterTask = workerContext.getQueryMaster().getQueryMasterTask(ebId.getQueryId());
+
+ if(queryMasterTask == null || queryMasterTask.isStopped()) {
+ done.run(DefaultTaskScheduler.stopTaskRunnerReq);
+ } else {
+ TajoContainerId cid =
+ queryMasterTask.getQueryTaskContext().getResourceAllocator().makeContainerId(request.getContainerId());
+ LOG.debug("getTask:" + cid + ", ebId:" + ebId);
+ queryMasterTask.handleTaskRequestEvent(new TaskRequestEvent(request.getWorkerId(), cid, ebId, done));
+ }
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ }
+
+ @Override
+ public void statusUpdate(RpcController controller, TajoWorkerProtocol.TaskStatusProto request,
+ RpcCallback<PrimitiveProtos.BoolProto> done) {
+ try {
+ QueryId queryId = new QueryId(request.getId().getTaskId().getExecutionBlockId().getQueryId());
+ TaskAttemptId attemptId = new TaskAttemptId(request.getId());
+ QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(queryId);
+ if (queryMasterTask == null) {
+ queryMasterTask = queryMaster.getQueryMasterTask(queryId, true);
+ }
+ Stage sq = queryMasterTask.getQuery().getStage(attemptId.getTaskId().getExecutionBlockId());
+ Task task = sq.getTask(attemptId.getTaskId());
+ TaskAttempt attempt = task.getAttempt(attemptId.getId());
+
+ if(LOG.isDebugEnabled()){
+ LOG.debug(String.format("Task State: %s, Attempt State: %s", task.getState().name(), attempt.getState().name()));
+ }
+
+ if (request.getState() == TajoProtos.TaskAttemptState.TA_KILLED) {
+ LOG.warn(attemptId + " Killed");
+ attempt.handle(
+ new TaskAttemptEvent(new TaskAttemptId(request.getId()), TaskAttemptEventType.TA_LOCAL_KILLED));
+ } else {
+ queryMasterTask.getEventHandler().handle(
+ new TaskAttemptStatusUpdateEvent(new TaskAttemptId(request.getId()), request));
+ }
+ done.run(TajoWorker.TRUE_PROTO);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ done.run(TajoWorker.FALSE_PROTO);
+ }
+ }
+
+ @Override
+ public void ping(RpcController controller,
+ TajoIdProtos.ExecutionBlockIdProto requestProto,
+ RpcCallback<PrimitiveProtos.BoolProto> done) {
+ done.run(TajoWorker.TRUE_PROTO);
+ }
+
+ @Override
+ public void fatalError(RpcController controller, TajoWorkerProtocol.TaskFatalErrorReport report,
+ RpcCallback<PrimitiveProtos.BoolProto> done) {
+ try {
+ QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(
+ new QueryId(report.getId().getTaskId().getExecutionBlockId().getQueryId()));
+ if (queryMasterTask != null) {
+ queryMasterTask.handleTaskFailed(report);
+ } else {
+ LOG.warn("No QueryMasterTask: " + new TaskAttemptId(report.getId()));
+ }
+ done.run(TajoWorker.TRUE_PROTO);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ done.run(TajoWorker.FALSE_PROTO);
+ }
+ }
+
+ @Override
+ public void done(RpcController controller, TajoWorkerProtocol.TaskCompletionReport report,
+ RpcCallback<PrimitiveProtos.BoolProto> done) {
+ try {
+ QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(
+ new QueryId(report.getId().getTaskId().getExecutionBlockId().getQueryId()));
+ if (queryMasterTask != null) {
+ queryMasterTask.getEventHandler().handle(new TaskCompletionEvent(report));
+ }
+ done.run(TajoWorker.TRUE_PROTO);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ done.run(TajoWorker.FALSE_PROTO);
+ }
+ }
+
+ @Override
+ public void doneExecutionBlock(
+ RpcController controller, TajoWorkerProtocol.ExecutionBlockReport request,
+ RpcCallback<PrimitiveProtos.BoolProto> done) {
+ QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(new QueryId(request.getEbId().getQueryId()));
+ if (queryMasterTask != null) {
+ ExecutionBlockId ebId = new ExecutionBlockId(request.getEbId());
+ queryMasterTask.getQuery().getStage(ebId).receiveExecutionBlockReport(request);
+ }
+ done.run(TajoWorker.TRUE_PROTO);
+ }
+
+ @Override
+ public void killQuery(RpcController controller, TajoIdProtos.QueryIdProto request,
+ RpcCallback<PrimitiveProtos.BoolProto> done) {
+ QueryId queryId = new QueryId(request);
+ QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(queryId);
+ if (queryMasterTask != null) {
+ Query query = queryMasterTask.getQuery();
+ if (query != null) {
+ query.handle(new QueryEvent(queryId, QueryEventType.KILL));
+ }
+ }
+ }
+
+ @Override
+ public void executeQuery(RpcController controller,
+ TajoWorkerProtocol.QueryExecutionRequestProto request,
+ RpcCallback<PrimitiveProtos.BoolProto> done) {
+ try {
+ workerContext.getWorkerSystemMetrics().counter("querymaster", "numQuery").inc();
+
+ QueryId queryId = new QueryId(request.getQueryId());
+ LOG.info("Receive executeQuery request:" + queryId);
+ queryMaster.handle(new QueryStartEvent(queryId,
+ new Session(request.getSession()),
+ new QueryContext(workerContext.getQueryMaster().getContext().getConf(),
+ request.getQueryContext()), request.getExprInJson().getValue(),
+ request.getLogicalPlanJson().getValue()));
+ done.run(TajoWorker.TRUE_PROTO);
+ } catch (Exception e) {
+ workerContext.getWorkerSystemMetrics().counter("querymaster", "errorQuery").inc();
+ LOG.error(e.getMessage(), e);
+ done.run(TajoWorker.FALSE_PROTO);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
new file mode 100644
index 0000000..bab5903
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
@@ -0,0 +1,638 @@
+/**
+ * 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.tajo.querymaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tajo.*;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.algebra.JsonHelper;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.exception.UnimplementedException;
+import org.apache.tajo.ha.HAServiceUtil;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.TajoContainerProxy;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.rm.TajoWorkerResourceManager;
+import org.apache.tajo.session.Session;
+import org.apache.tajo.plan.LogicalOptimizer;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.LogicalPlanner;
+import org.apache.tajo.plan.logical.LogicalNode;
+import org.apache.tajo.plan.logical.LogicalRootNode;
+import org.apache.tajo.plan.logical.NodeType;
+import org.apache.tajo.plan.logical.ScanNode;
+import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRule;
+import org.apache.tajo.plan.util.PlannerUtil;
+import org.apache.tajo.plan.verifier.VerifyException;
+import org.apache.tajo.rpc.NettyClientBase;
+import org.apache.tajo.rpc.RpcConnectionPool;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.StorageProperty;
+import org.apache.tajo.storage.StorageUtil;
+import org.apache.tajo.util.metrics.TajoMetrics;
+import org.apache.tajo.util.metrics.reporter.MetricsConsoleReporter;
+import org.apache.tajo.worker.AbstractResourceAllocator;
+import org.apache.tajo.worker.TajoResourceAllocator;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.tajo.TajoProtos.QueryState;
+
+public class QueryMasterTask extends CompositeService {
+ private static final Log LOG = LogFactory.getLog(QueryMasterTask.class.getName());
+
+ // query submission directory is private!
+ final public static FsPermission STAGING_DIR_PERMISSION =
+ FsPermission.createImmutable((short) 0700); // rwx--------
+
+ public static final String TMP_STAGING_DIR_PREFIX = ".staging";
+
+ private QueryId queryId;
+
+ private Session session;
+
+ private QueryContext queryContext;
+
+ private QueryMasterTaskContext queryTaskContext;
+
+ private QueryMaster.QueryMasterContext queryMasterContext;
+
+ private Query query;
+
+ private MasterPlan masterPlan;
+
+ private String jsonExpr;
+
+ private String logicalPlanJson;
+
+ private AsyncDispatcher dispatcher;
+
+ private final long querySubmitTime;
+
+ private Map<String, TableDesc> tableDescMap = new HashMap<String, TableDesc>();
+
+ private TajoConf systemConf;
+
+ private AtomicLong lastClientHeartbeat = new AtomicLong(-1);
+
+ private AbstractResourceAllocator resourceAllocator;
+
+ private AtomicBoolean stopped = new AtomicBoolean(false);
+
+ private TajoMetrics queryMetrics;
+
+ private Throwable initError;
+
+ private final List<TajoWorkerProtocol.TaskFatalErrorReport> diagnostics =
+ new ArrayList<TajoWorkerProtocol.TaskFatalErrorReport>();
+
+ public QueryMasterTask(QueryMaster.QueryMasterContext queryMasterContext,
+ QueryId queryId, Session session, QueryContext queryContext, String jsonExpr,
+ String logicalPlanJson) {
+
+ super(QueryMasterTask.class.getName());
+ this.queryMasterContext = queryMasterContext;
+ this.queryId = queryId;
+ this.session = session;
+ this.queryContext = queryContext;
+ this.jsonExpr = jsonExpr;
+ this.logicalPlanJson = logicalPlanJson;
+ this.querySubmitTime = System.currentTimeMillis();
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ systemConf = (TajoConf)conf;
+
+ try {
+ queryTaskContext = new QueryMasterTaskContext();
+ String resourceManagerClassName = systemConf.getVar(TajoConf.ConfVars.RESOURCE_MANAGER_CLASS);
+
+ if(resourceManagerClassName.indexOf(TajoWorkerResourceManager.class.getName()) >= 0) {
+ resourceAllocator = new TajoResourceAllocator(queryTaskContext);
+ } else {
+ throw new UnimplementedException(resourceManagerClassName + " is not supported yet");
+ }
+ addService(resourceAllocator);
+
+ dispatcher = new AsyncDispatcher();
+ addService(dispatcher);
+
+ dispatcher.register(StageEventType.class, new StageEventDispatcher());
+ dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
+ dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDispatcher());
+ dispatcher.register(QueryMasterQueryCompletedEvent.EventType.class, new QueryFinishEventHandler());
+ dispatcher.register(TaskSchedulerEvent.EventType.class, new TaskSchedulerDispatcher());
+ dispatcher.register(LocalTaskEventType.class, new LocalTaskEventHandler());
+
+ initStagingDir();
+
+ queryMetrics = new TajoMetrics(queryId.toString());
+
+ super.init(systemConf);
+ } catch (Throwable t) {
+ LOG.error(t.getMessage(), t);
+ initError = t;
+ }
+ }
+
+ public boolean isStopped() {
+ return stopped.get();
+ }
+
+ @Override
+ public void start() {
+ startQuery();
+ super.start();
+ }
+
+ @Override
+ public void stop() {
+
+ if(stopped.getAndSet(true)) {
+ return;
+ }
+
+ LOG.info("Stopping QueryMasterTask:" + queryId);
+
+ try {
+ resourceAllocator.stop();
+ } catch (Throwable t) {
+ LOG.fatal(t.getMessage(), t);
+ }
+
+ RpcConnectionPool connPool = RpcConnectionPool.getPool(queryMasterContext.getConf());
+ NettyClientBase tmClient = null;
+ try {
+ // In TajoMaster HA mode, if backup master be active status,
+ // worker may fail to connect existing active master. Thus,
+ // if worker can't connect the master, worker should try to connect another master and
+ // update master address in worker context.
+ if (systemConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
+ try {
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ } catch (Exception e) {
+ queryMasterContext.getWorkerContext().setWorkerResourceTrackerAddr(
+ HAServiceUtil.getResourceTrackerAddress(systemConf));
+ queryMasterContext.getWorkerContext().setTajoMasterAddress(
+ HAServiceUtil.getMasterUmbilicalAddress(systemConf));
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ }
+ } else {
+ tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
+ TajoMasterProtocol.class, true);
+ }
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ } finally {
+ connPool.releaseConnection(tmClient);
+ }
+
+ super.stop();
+
+ //TODO change report to tajo master
+ if (queryMetrics != null) {
+ queryMetrics.report(new MetricsConsoleReporter());
+ }
+
+ LOG.info("Stopped QueryMasterTask:" + queryId);
+ }
+
+ public void handleTaskRequestEvent(TaskRequestEvent event) {
+ ExecutionBlockId id = event.getExecutionBlockId();
+ query.getStage(id).handleTaskRequestEvent(event);
+ }
+
+ public void handleTaskFailed(TajoWorkerProtocol.TaskFatalErrorReport report) {
+ synchronized(diagnostics) {
+ if (diagnostics.size() < 10) {
+ diagnostics.add(report);
+ }
+ }
+
+ getEventHandler().handle(new TaskFatalErrorEvent(report));
+ }
+
+ public Collection<TajoWorkerProtocol.TaskFatalErrorReport> getDiagnostics() {
+ synchronized(diagnostics) {
+ return Collections.unmodifiableCollection(diagnostics);
+ }
+ }
+
+ private class StageEventDispatcher implements EventHandler<StageEvent> {
+ public void handle(StageEvent event) {
+ ExecutionBlockId id = event.getStageId();
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("StageEventDispatcher:" + id + "," + event.getType());
+ }
+ query.getStage(id).handle(event);
+ }
+ }
+
+ private class TaskEventDispatcher
+ implements EventHandler<TaskEvent> {
+ public void handle(TaskEvent event) {
+ TaskId taskId = event.getTaskId();
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("TaskEventDispatcher>" + taskId + "," + event.getType());
+ }
+ Task task = query.getStage(taskId.getExecutionBlockId()).
+ getTask(taskId);
+ task.handle(event);
+ }
+ }
+
+ private class TaskAttemptEventDispatcher
+ implements EventHandler<TaskAttemptEvent> {
+ public void handle(TaskAttemptEvent event) {
+ TaskAttemptId attemptId = event.getTaskAttemptId();
+ Stage stage = query.getStage(attemptId.getTaskId().getExecutionBlockId());
+ Task task = stage.getTask(attemptId.getTaskId());
+ TaskAttempt attempt = task.getAttempt(attemptId);
+ attempt.handle(event);
+ }
+ }
+
+ private class TaskSchedulerDispatcher
+ implements EventHandler<TaskSchedulerEvent> {
+ public void handle(TaskSchedulerEvent event) {
+ Stage stage = query.getStage(event.getExecutionBlockId());
+ stage.getTaskScheduler().handle(event);
+ }
+ }
+
+ private class LocalTaskEventHandler implements EventHandler<LocalTaskEvent> {
+ @Override
+ public void handle(LocalTaskEvent event) {
+ TajoContainerProxy proxy = (TajoContainerProxy) resourceAllocator.getContainers().get(event.getContainerId());
+ if (proxy != null) {
+ proxy.killTaskAttempt(event.getTaskAttemptId());
+ }
+ }
+ }
+
+ private class QueryFinishEventHandler implements EventHandler<QueryMasterQueryCompletedEvent> {
+ @Override
+ public void handle(QueryMasterQueryCompletedEvent event) {
+ QueryId queryId = event.getQueryId();
+ LOG.info("Query completion notified from " + queryId);
+
+ while (!isTerminatedState(query.getSynchronizedState())) {
+ try {
+ synchronized (this) {
+ wait(10);
+ }
+ } catch (InterruptedException e) {
+ LOG.error(e);
+ }
+ }
+ LOG.info("Query final state: " + query.getSynchronizedState());
+
+ queryMasterContext.getEventHandler().handle(new QueryStopEvent(queryId));
+ }
+ }
+
+ private static boolean isTerminatedState(QueryState state) {
+ return
+ state == QueryState.QUERY_SUCCEEDED ||
+ state == QueryState.QUERY_FAILED ||
+ state == QueryState.QUERY_KILLED ||
+ state == QueryState.QUERY_ERROR;
+ }
+
+ public synchronized void startQuery() {
+ StorageManager sm = null;
+ LogicalPlan plan = null;
+ try {
+ if (query != null) {
+ LOG.warn("Query already started");
+ return;
+ }
+ CatalogService catalog = getQueryTaskContext().getQueryMasterContext().getWorkerContext().getCatalog();
+ LogicalPlanner planner = new LogicalPlanner(catalog);
+ LogicalOptimizer optimizer = new LogicalOptimizer(systemConf);
+ Expr expr = JsonHelper.fromJson(jsonExpr, Expr.class);
+ jsonExpr = null; // remove the possible OOM
+ plan = planner.createPlan(queryContext, expr);
+
+ StoreType storeType = PlannerUtil.getStoreType(plan);
+ if (storeType != null) {
+ sm = StorageManager.getStorageManager(systemConf, storeType);
+ StorageProperty storageProperty = sm.getStorageProperty();
+ if (storageProperty.isSortedInsert()) {
+ String tableName = PlannerUtil.getStoreTableName(plan);
+ LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+ TableDesc tableDesc = PlannerUtil.getTableDesc(catalog, rootNode.getChild());
+ if (tableDesc == null) {
+ throw new VerifyException("Can't get table meta data from catalog: " + tableName);
+ }
+ List<LogicalPlanRewriteRule> storageSpecifiedRewriteRules = sm.getRewriteRules(
+ getQueryTaskContext().getQueryContext(), tableDesc);
+ if (storageSpecifiedRewriteRules != null) {
+ for (LogicalPlanRewriteRule eachRule: storageSpecifiedRewriteRules) {
+ optimizer.addRuleAfterToJoinOpt(eachRule);
+ }
+ }
+ }
+ }
+
+ optimizer.optimize(queryContext, plan);
+
+ for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
+ LogicalNode[] scanNodes = PlannerUtil.findAllNodes(block.getRoot(), NodeType.SCAN);
+ if (scanNodes != null) {
+ for (LogicalNode eachScanNode : scanNodes) {
+ ScanNode scanNode = (ScanNode) eachScanNode;
+ tableDescMap.put(scanNode.getCanonicalName(), scanNode.getTableDesc());
+ }
+ }
+
+ scanNodes = PlannerUtil.findAllNodes(block.getRoot(), NodeType.PARTITIONS_SCAN);
+ if (scanNodes != null) {
+ for (LogicalNode eachScanNode : scanNodes) {
+ ScanNode scanNode = (ScanNode) eachScanNode;
+ tableDescMap.put(scanNode.getCanonicalName(), scanNode.getTableDesc());
+ }
+ }
+ }
+ MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
+ queryMasterContext.getGlobalPlanner().build(masterPlan);
+
+ query = new Query(queryTaskContext, queryId, querySubmitTime,
+ "", queryTaskContext.getEventHandler(), masterPlan);
+
+ dispatcher.register(QueryEventType.class, query);
+ queryTaskContext.getEventHandler().handle(new QueryEvent(queryId, QueryEventType.START));
+ } catch (Throwable t) {
+ LOG.error(t.getMessage(), t);
+ initError = t;
+
+ if (plan != null && sm != null) {
+ LogicalRootNode rootNode = plan.getRootBlock().getRoot();
+ try {
+ sm.rollbackOutputCommit(rootNode.getChild());
+ } catch (IOException e) {
+ LOG.warn(query.getId() + ", failed processing cleanup storage when query failed:" + e.getMessage(), e);
+ }
+ }
+ }
+ }
+
+ private void initStagingDir() throws IOException {
+ Path stagingDir = null;
+ FileSystem defaultFS = TajoConf.getWarehouseDir(systemConf).getFileSystem(systemConf);
+
+ try {
+
+ stagingDir = initStagingDir(systemConf, queryId.toString(), queryContext);
+
+ // Create a subdirectories
+ LOG.info("The staging dir '" + stagingDir + "' is created.");
+ queryContext.setStagingDir(stagingDir);
+ } catch (IOException ioe) {
+ if (stagingDir != null && defaultFS.exists(stagingDir)) {
+ try {
+ defaultFS.delete(stagingDir, true);
+ LOG.info("The staging directory '" + stagingDir + "' is deleted");
+ } catch (Exception e) {
+ LOG.warn(e.getMessage());
+ }
+ }
+
+ throw ioe;
+ }
+ }
+
+ /**
+ * It initializes the final output and staging directory and sets
+ * them to variables.
+ */
+ public static Path initStagingDir(TajoConf conf, String queryId, QueryContext context) throws IOException {
+
+ String realUser;
+ String currentUser;
+ UserGroupInformation ugi;
+ ugi = UserGroupInformation.getLoginUser();
+ realUser = ugi.getShortUserName();
+ currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
+
+ FileSystem fs;
+ Path stagingDir;
+
+ ////////////////////////////////////////////
+ // Create Output Directory
+ ////////////////////////////////////////////
+
+ String outputPath = context.get(QueryVars.OUTPUT_TABLE_PATH, "");
+ if (context.isCreateTable() || context.isInsert()) {
+ if (outputPath == null || outputPath.isEmpty()) {
+ // hbase
+ stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
+ } else {
+ stagingDir = StorageUtil.concatPath(context.getOutputPath(), TMP_STAGING_DIR_PREFIX, queryId);
+ }
+ } else {
+ stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
+ }
+
+ // initializ
+ fs = stagingDir.getFileSystem(conf);
+
+ if (fs.exists(stagingDir)) {
+ throw new IOException("The staging directory '" + stagingDir + "' already exists");
+ }
+ fs.mkdirs(stagingDir, new FsPermission(STAGING_DIR_PERMISSION));
+ FileStatus fsStatus = fs.getFileStatus(stagingDir);
+ String owner = fsStatus.getOwner();
+
+ if (!owner.isEmpty() && !(owner.equals(currentUser) || owner.equals(realUser))) {
+ throw new IOException("The ownership on the user's query " +
+ "directory " + stagingDir + " is not as expected. " +
+ "It is owned by " + owner + ". The directory must " +
+ "be owned by the submitter " + currentUser + " or " +
+ "by " + realUser);
+ }
+
+ if (!fsStatus.getPermission().equals(STAGING_DIR_PERMISSION)) {
+ LOG.info("Permissions on staging directory " + stagingDir + " are " +
+ "incorrect: " + fsStatus.getPermission() + ". Fixing permissions " +
+ "to correct value " + STAGING_DIR_PERMISSION);
+ fs.setPermission(stagingDir, new FsPermission(STAGING_DIR_PERMISSION));
+ }
+
+ Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
+ fs.mkdirs(stagingResultDir);
+
+ return stagingDir;
+ }
+
+ public Query getQuery() {
+ return query;
+ }
+
+ protected void expireQuerySession() {
+ if(!isTerminatedState(query.getState()) && !(query.getState() == QueryState.QUERY_KILL_WAIT)){
+ query.handle(new QueryEvent(queryId, QueryEventType.KILL));
+ }
+ }
+
+ public QueryMasterTaskContext getQueryTaskContext() {
+ return queryTaskContext;
+ }
+
+ public EventHandler getEventHandler() {
+ return queryTaskContext.getEventHandler();
+ }
+
+ public void touchSessionTime() {
+ this.lastClientHeartbeat.set(System.currentTimeMillis());
+ }
+
+ public long getLastClientHeartbeat() {
+ return this.lastClientHeartbeat.get();
+ }
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public boolean isInitError() {
+ return initError != null;
+ }
+
+ public QueryState getState() {
+ if(query == null) {
+ if (isInitError()) {
+ return QueryState.QUERY_ERROR;
+ } else {
+ return QueryState.QUERY_NOT_ASSIGNED;
+ }
+ } else {
+ return query.getState();
+ }
+ }
+
+ public Throwable getInitError() {
+ return initError;
+ }
+
+ public String getErrorMessage() {
+ if (isInitError()) {
+ return StringUtils.stringifyException(initError);
+ } else {
+ return null;
+ }
+ }
+
+ public long getQuerySubmitTime() {
+ return this.querySubmitTime;
+ }
+
+ public class QueryMasterTaskContext {
+ EventHandler eventHandler;
+ public QueryMaster.QueryMasterContext getQueryMasterContext() {
+ return queryMasterContext;
+ }
+
+ public Session getSession() {
+ return session;
+ }
+
+ public QueryContext getQueryContext() {
+ return queryContext;
+ }
+
+ public TajoConf getConf() {
+ return systemConf;
+ }
+
+ public Clock getClock() {
+ return queryMasterContext.getClock();
+ }
+
+ public Query getQuery() {
+ return query;
+ }
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public Path getStagingDir() {
+ return queryContext.getStagingDir();
+ }
+
+ public synchronized EventHandler getEventHandler() {
+ if(eventHandler == null) {
+ eventHandler = dispatcher.getEventHandler();
+ }
+ return eventHandler;
+ }
+
+ public AsyncDispatcher getDispatcher() {
+ return dispatcher;
+ }
+
+ public Stage getStage(ExecutionBlockId id) {
+ return query.getStage(id);
+ }
+
+ public Map<String, TableDesc> getTableDescMap() {
+ return tableDescMap;
+ }
+
+ public float getProgress() {
+ if(query == null) {
+ return 0.0f;
+ }
+ return query.getProgress();
+ }
+
+ public AbstractResourceAllocator getResourceAllocator() {
+ return resourceAllocator;
+ }
+
+ public TajoMetrics getQueryMetrics() {
+ return queryMetrics;
+ }
+ }
+}
\ No newline at end of file
[05/16] tajo git commit: TAJO-1288: Refactoring
org.apache.tajo.master package.
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
new file mode 100644
index 0000000..5a35674
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Repartitioner.java
@@ -0,0 +1,1250 @@
+/**
+ * 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.tajo.querymaster;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.algebra.JoinType;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.statistics.StatisticsUtil;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf.ConfVars;
+import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
+import org.apache.tajo.engine.planner.RangePartitionAlgorithm;
+import org.apache.tajo.engine.planner.UniformRangePartition;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.global.DataChannel;
+import org.apache.tajo.engine.planner.global.ExecutionBlock;
+import org.apache.tajo.engine.planner.global.GlobalPlanner;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.utils.TupleUtil;
+import org.apache.tajo.exception.InternalException;
+import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.MultipleAggregationStage;
+import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty;
+import org.apache.tajo.querymaster.Task.IntermediateEntry;
+import org.apache.tajo.plan.logical.SortNode.SortPurpose;
+import org.apache.tajo.plan.util.PlannerUtil;
+import org.apache.tajo.plan.PlanningException;
+import org.apache.tajo.plan.logical.*;
+import org.apache.tajo.storage.FileStorageManager;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.RowStoreUtil;
+import org.apache.tajo.storage.TupleRange;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.util.Pair;
+import org.apache.tajo.unit.StorageUnit;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.util.TajoIdUtils;
+import org.apache.tajo.worker.FetchImpl;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.math.BigInteger;
+import java.net.URI;
+import java.util.*;
+import java.util.Map.Entry;
+
+import static org.apache.tajo.plan.serder.PlanProto.ShuffleType;
+import static org.apache.tajo.plan.serder.PlanProto.ShuffleType.*;
+
+/**
+ * Repartitioner creates non-leaf tasks and shuffles intermediate data.
+ * It supports two repartition methods, such as hash and range repartition.
+ */
+public class Repartitioner {
+ private static final Log LOG = LogFactory.getLog(Repartitioner.class);
+
+ private final static int HTTP_REQUEST_MAXIMUM_LENGTH = 1900;
+ private final static String UNKNOWN_HOST = "unknown";
+
+ public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerContext, Stage stage)
+ throws IOException {
+ MasterPlan masterPlan = stage.getMasterPlan();
+ ExecutionBlock execBlock = stage.getBlock();
+ QueryMasterTask.QueryMasterTaskContext masterContext = stage.getContext();
+
+ ScanNode[] scans = execBlock.getScanNodes();
+
+ Path tablePath;
+ Fragment[] fragments = new Fragment[scans.length];
+ long[] stats = new long[scans.length];
+
+ // initialize variables from the child operators
+ for (int i = 0; i < scans.length; i++) {
+ TableDesc tableDesc = masterContext.getTableDescMap().get(scans[i].getCanonicalName());
+ if (tableDesc == null) { // if it is a real table stored on storage
+ FileStorageManager storageManager =
+ (FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf());
+
+ tablePath = storageManager.getTablePath(scans[i].getTableName());
+ if (execBlock.getUnionScanMap() != null && !execBlock.getUnionScanMap().isEmpty()) {
+ for (Map.Entry<ExecutionBlockId, ExecutionBlockId> unionScanEntry: execBlock.getUnionScanMap().entrySet()) {
+ ExecutionBlockId originScanEbId = unionScanEntry.getKey();
+ stats[i] += masterContext.getStage(originScanEbId).getResultStats().getNumBytes();
+ }
+ } else {
+ ExecutionBlockId scanEBId = TajoIdUtils.createExecutionBlockId(scans[i].getTableName());
+ stats[i] = masterContext.getStage(scanEBId).getResultStats().getNumBytes();
+ }
+ fragments[i] = new FileFragment(scans[i].getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
+ } else {
+ try {
+ stats[i] = GlobalPlanner.computeDescendentVolume(scans[i]);
+ } catch (PlanningException e) {
+ throw new IOException(e);
+ }
+
+ StorageManager storageManager =
+ StorageManager.getStorageManager(stage.getContext().getConf(), tableDesc.getMeta().getStoreType());
+
+ // if table has no data, storageManager will return empty FileFragment.
+ // So, we need to handle FileFragment by its size.
+ // If we don't check its size, it can cause IndexOutOfBoundsException.
+ List<Fragment> fileFragments = storageManager.getSplits(scans[i].getCanonicalName(), tableDesc);
+ if (fileFragments.size() > 0) {
+ fragments[i] = fileFragments.get(0);
+ } else {
+ fragments[i] = new FileFragment(scans[i].getCanonicalName(), new Path(tableDesc.getPath()), 0, 0, new String[]{UNKNOWN_HOST});
+ }
+ }
+ }
+
+ // If one of inner join tables has no input data, it means that this execution block has no result row.
+ JoinNode joinNode = PlannerUtil.findMostBottomNode(execBlock.getPlan(), NodeType.JOIN);
+ if (joinNode != null) {
+ if ( (joinNode.getJoinType() == JoinType.INNER)) {
+ LogicalNode leftNode = joinNode.getLeftChild();
+ LogicalNode rightNode = joinNode.getRightChild();
+ for (int i = 0; i < stats.length; i++) {
+ if (scans[i].getPID() == leftNode.getPID() || scans[i].getPID() == rightNode.getPID()) {
+ if (stats[i] == 0) {
+ LOG.info(scans[i] + " 's input data is zero. Inner join's result is empty.");
+ return;
+ }
+ }
+ }
+ }
+ }
+
+ // If node is outer join and a preserved relation is empty, it should return zero rows.
+ joinNode = PlannerUtil.findTopNode(execBlock.getPlan(), NodeType.JOIN);
+ if (joinNode != null) {
+ // If all stats are zero, return
+ boolean isEmptyAllJoinTables = true;
+ for (int i = 0; i < stats.length; i++) {
+ if (stats[i] > 0) {
+ isEmptyAllJoinTables = false;
+ break;
+ }
+ }
+ if (isEmptyAllJoinTables) {
+ LOG.info("All input join tables are empty.");
+ return;
+ }
+
+ // find left top scan node
+ ScanNode leftScanNode = PlannerUtil.findTopNode(joinNode.getLeftChild(), NodeType.SCAN);
+ ScanNode rightScanNode = PlannerUtil.findTopNode(joinNode.getRightChild(), NodeType.SCAN);
+
+ long leftStats = -1;
+ long rightStats = -1;
+ if (stats.length == 2) {
+ for (int i = 0; i < stats.length; i++) {
+ if (scans[i].equals(leftScanNode)) {
+ leftStats = stats[i];
+ } else if (scans[i].equals(rightScanNode)) {
+ rightStats = stats[i];
+ }
+ }
+ if (joinNode.getJoinType() == JoinType.LEFT_OUTER) {
+ if (leftStats == 0) {
+ return;
+ }
+ }
+ if (joinNode.getJoinType() == JoinType.RIGHT_OUTER) {
+ if (rightStats == 0) {
+ return;
+ }
+ }
+ }
+ }
+
+ // Assigning either fragments or fetch urls to query units
+ boolean isAllBroadcastTable = true;
+ for (int i = 0; i < scans.length; i++) {
+ if (!execBlock.isBroadcastTable(scans[i].getCanonicalName())) {
+ isAllBroadcastTable = false;
+ break;
+ }
+ }
+
+
+ if (isAllBroadcastTable) { // if all relations of this EB are broadcasted
+ // set largest table to normal mode
+ long maxStats = Long.MIN_VALUE;
+ int maxStatsScanIdx = -1;
+ for (int i = 0; i < scans.length; i++) {
+ // finding largest table.
+ // If stats == 0, can't be base table.
+ if (stats[i] > 0 && stats[i] > maxStats) {
+ maxStats = stats[i];
+ maxStatsScanIdx = i;
+ }
+ }
+ if (maxStatsScanIdx == -1) {
+ maxStatsScanIdx = 0;
+ }
+ int baseScanIdx = maxStatsScanIdx;
+ scans[baseScanIdx].setBroadcastTable(false);
+ execBlock.removeBroadcastTable(scans[baseScanIdx].getCanonicalName());
+ LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join with all tables, base_table=%s, base_volume=%d",
+ scans[baseScanIdx].getCanonicalName(), stats[baseScanIdx]));
+ scheduleLeafTasksWithBroadcastTable(schedulerContext, stage, baseScanIdx, fragments);
+ } else if (!execBlock.getBroadcastTables().isEmpty()) { // If some relations of this EB are broadcasted
+ boolean hasNonLeafNode = false;
+ List<Integer> largeScanIndexList = new ArrayList<Integer>();
+ List<Integer> broadcastIndexList = new ArrayList<Integer>();
+ String nonLeafScanNames = "";
+ String namePrefix = "";
+ long maxStats = Long.MIN_VALUE;
+ int maxStatsScanIdx = -1;
+ for (int i = 0; i < scans.length; i++) {
+ if (scans[i].getTableDesc().getMeta().getStoreType() == StoreType.RAW) {
+ // Intermediate data scan
+ hasNonLeafNode = true;
+ largeScanIndexList.add(i);
+ nonLeafScanNames += namePrefix + scans[i].getCanonicalName();
+ namePrefix = ",";
+ }
+ if (execBlock.isBroadcastTable(scans[i].getCanonicalName())) {
+ broadcastIndexList.add(i);
+ } else {
+ // finding largest table.
+ if (stats[i] > 0 && stats[i] > maxStats) {
+ maxStats = stats[i];
+ maxStatsScanIdx = i;
+ }
+ }
+ }
+ if (maxStatsScanIdx == -1) {
+ maxStatsScanIdx = 0;
+ }
+
+ if (!hasNonLeafNode) {
+ if (largeScanIndexList.size() > 1) {
+ String largeTableNames = "";
+ for (Integer eachId : largeScanIndexList) {
+ largeTableNames += scans[eachId].getTableName() + ",";
+ }
+ throw new IOException("Broadcast join with leaf node should have only one large table, " +
+ "but " + largeScanIndexList.size() + ", tables=" + largeTableNames);
+ }
+ int baseScanIdx = largeScanIndexList.isEmpty() ? maxStatsScanIdx : largeScanIndexList.get(0);
+ LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join, base_table=%s, base_volume=%d",
+ scans[baseScanIdx].getCanonicalName(), stats[baseScanIdx]));
+ scheduleLeafTasksWithBroadcastTable(schedulerContext, stage, baseScanIdx, fragments);
+ } else {
+ if (largeScanIndexList.size() > 2) {
+ throw new IOException("Symmetric Repartition Join should have two scan node, but " + nonLeafScanNames);
+ }
+
+ //select intermediate scan and stats
+ ScanNode[] intermediateScans = new ScanNode[largeScanIndexList.size()];
+ long[] intermediateScanStats = new long[largeScanIndexList.size()];
+ Fragment[] intermediateFragments = new Fragment[largeScanIndexList.size()];
+ int index = 0;
+ for (Integer eachIdx : largeScanIndexList) {
+ intermediateScans[index] = scans[eachIdx];
+ intermediateScanStats[index] = stats[eachIdx];
+ intermediateFragments[index++] = fragments[eachIdx];
+ }
+ Fragment[] broadcastFragments = new Fragment[broadcastIndexList.size()];
+ ScanNode[] broadcastScans = new ScanNode[broadcastIndexList.size()];
+ index = 0;
+ for (Integer eachIdx : broadcastIndexList) {
+ scans[eachIdx].setBroadcastTable(true);
+ broadcastScans[index] = scans[eachIdx];
+ broadcastFragments[index] = fragments[eachIdx];
+ index++;
+ }
+ LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join, join_node=%s", nonLeafScanNames));
+ scheduleSymmetricRepartitionJoin(masterContext, schedulerContext, stage,
+ intermediateScans, intermediateScanStats, intermediateFragments, broadcastScans, broadcastFragments);
+ }
+ } else {
+ LOG.info("[Distributed Join Strategy] : Symmetric Repartition Join");
+ scheduleSymmetricRepartitionJoin(masterContext, schedulerContext, stage, scans, stats, fragments, null, null);
+ }
+ }
+
+ /**
+ * Scheduling in tech case of Symmetric Repartition Join
+ * @param masterContext
+ * @param schedulerContext
+ * @param stage
+ * @param scans
+ * @param stats
+ * @param fragments
+ * @throws IOException
+ */
+ private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMasterTaskContext masterContext,
+ TaskSchedulerContext schedulerContext,
+ Stage stage,
+ ScanNode[] scans,
+ long[] stats,
+ Fragment[] fragments,
+ ScanNode[] broadcastScans,
+ Fragment[] broadcastFragments) throws IOException {
+ MasterPlan masterPlan = stage.getMasterPlan();
+ ExecutionBlock execBlock = stage.getBlock();
+ // The hash map is modeling as follows:
+ // <Part Id, <EbId, List<Intermediate Data>>>
+ Map<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>> hashEntries =
+ new HashMap<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>>();
+
+ // Grouping IntermediateData by a partition key and a table name
+ List<ExecutionBlock> childBlocks = masterPlan.getChilds(stage.getId());
+
+ // In the case of join with union, there is one ScanNode for union.
+ Map<ExecutionBlockId, ExecutionBlockId> unionScanMap = execBlock.getUnionScanMap();
+ for (ExecutionBlock childBlock : childBlocks) {
+ ExecutionBlockId scanEbId = unionScanMap.get(childBlock.getId());
+ if (scanEbId == null) {
+ scanEbId = childBlock.getId();
+ }
+ Stage childExecSM = stage.getContext().getStage(childBlock.getId());
+
+ if (childExecSM.getHashShuffleIntermediateEntries() != null &&
+ !childExecSM.getHashShuffleIntermediateEntries().isEmpty()) {
+ for (IntermediateEntry intermediateEntry: childExecSM.getHashShuffleIntermediateEntries()) {
+ intermediateEntry.setEbId(childBlock.getId());
+ if (hashEntries.containsKey(intermediateEntry.getPartId())) {
+ Map<ExecutionBlockId, List<IntermediateEntry>> tbNameToInterm =
+ hashEntries.get(intermediateEntry.getPartId());
+
+ if (tbNameToInterm.containsKey(scanEbId)) {
+ tbNameToInterm.get(scanEbId).add(intermediateEntry);
+ } else {
+ tbNameToInterm.put(scanEbId, TUtil.newList(intermediateEntry));
+ }
+ } else {
+ Map<ExecutionBlockId, List<IntermediateEntry>> tbNameToInterm =
+ new HashMap<ExecutionBlockId, List<IntermediateEntry>>();
+ tbNameToInterm.put(scanEbId, TUtil.newList(intermediateEntry));
+ hashEntries.put(intermediateEntry.getPartId(), tbNameToInterm);
+ }
+ }
+ } else {
+ //if no intermidatedata(empty table), make empty entry
+ int emptyPartitionId = 0;
+ if (hashEntries.containsKey(emptyPartitionId)) {
+ Map<ExecutionBlockId, List<IntermediateEntry>> tbNameToInterm = hashEntries.get(emptyPartitionId);
+ if (tbNameToInterm.containsKey(scanEbId))
+ tbNameToInterm.get(scanEbId).addAll(new ArrayList<IntermediateEntry>());
+ else
+ tbNameToInterm.put(scanEbId, new ArrayList<IntermediateEntry>());
+ } else {
+ Map<ExecutionBlockId, List<IntermediateEntry>> tbNameToInterm =
+ new HashMap<ExecutionBlockId, List<IntermediateEntry>>();
+ tbNameToInterm.put(scanEbId, new ArrayList<IntermediateEntry>());
+ hashEntries.put(emptyPartitionId, tbNameToInterm);
+ }
+ }
+ }
+
+ // hashEntries can be zero if there are no input data.
+ // In the case, it will cause the zero divided exception.
+ // it avoids this problem.
+ int[] avgSize = new int[2];
+ avgSize[0] = hashEntries.size() == 0 ? 0 : (int) (stats[0] / hashEntries.size());
+ avgSize[1] = hashEntries.size() == 0 ? 0 : (int) (stats[1] / hashEntries.size());
+ int bothFetchSize = avgSize[0] + avgSize[1];
+
+ // Getting the desire number of join tasks according to the volumn
+ // of a larger table
+ int largerIdx = stats[0] >= stats[1] ? 0 : 1;
+ int desireJoinTaskVolumn = stage.getMasterPlan().getContext().getInt(SessionVars.JOIN_TASK_INPUT_SIZE);
+
+ // calculate the number of tasks according to the data size
+ int mb = (int) Math.ceil((double) stats[largerIdx] / 1048576);
+ LOG.info("Larger intermediate data is approximately " + mb + " MB");
+ // determine the number of task per 64MB
+ int maxTaskNum = (int) Math.ceil((double) mb / desireJoinTaskVolumn);
+ LOG.info("The calculated number of tasks is " + maxTaskNum);
+ LOG.info("The number of total shuffle keys is " + hashEntries.size());
+ // the number of join tasks cannot be larger than the number of
+ // distinct partition ids.
+ int joinTaskNum = Math.min(maxTaskNum, hashEntries.size());
+ LOG.info("The determined number of join tasks is " + joinTaskNum);
+
+ List<Fragment> rightFragments = new ArrayList<Fragment>();
+ rightFragments.add(fragments[1]);
+
+ if (broadcastFragments != null) {
+ //In this phase a ScanNode has a single fragment.
+ //If there are more than one data files, that files should be added to fragments or partition path
+ for (ScanNode eachScan: broadcastScans) {
+ Path[] partitionScanPaths = null;
+ TableDesc tableDesc = masterContext.getTableDescMap().get(eachScan.getCanonicalName());
+ if (eachScan.getType() == NodeType.PARTITIONS_SCAN) {
+ FileStorageManager storageManager =
+ (FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf());
+
+ PartitionedTableScanNode partitionScan = (PartitionedTableScanNode)eachScan;
+ partitionScanPaths = partitionScan.getInputPaths();
+ // set null to inputPaths in getFragmentsFromPartitionedTable()
+ getFragmentsFromPartitionedTable(storageManager, eachScan, tableDesc);
+ partitionScan.setInputPaths(partitionScanPaths);
+ } else {
+ StorageManager storageManager = StorageManager.getStorageManager(stage.getContext().getConf(),
+ tableDesc.getMeta().getStoreType());
+ Collection<Fragment> scanFragments = storageManager.getSplits(eachScan.getCanonicalName(),
+ tableDesc, eachScan);
+ if (scanFragments != null) {
+ rightFragments.addAll(scanFragments);
+ }
+ }
+ }
+ }
+ Stage.scheduleFragment(stage, fragments[0], rightFragments);
+
+ // Assign partitions to tasks in a round robin manner.
+ for (Entry<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>> entry
+ : hashEntries.entrySet()) {
+ addJoinShuffle(stage, entry.getKey(), entry.getValue());
+ }
+
+ schedulerContext.setTaskSize((int) Math.ceil((double) bothFetchSize / joinTaskNum));
+ schedulerContext.setEstimatedTaskNum(joinTaskNum);
+ }
+
+ /**
+ * merge intermediate entry by ebid, pullhost
+ * @param hashEntries
+ * @return
+ */
+ public static Map<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>> mergeIntermediateByPullHost(
+ Map<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>> hashEntries) {
+ Map<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>> mergedHashEntries =
+ new HashMap<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>>();
+
+ for(Entry<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>> entry: hashEntries.entrySet()) {
+ Integer partId = entry.getKey();
+ for (Entry<ExecutionBlockId, List<IntermediateEntry>> partEntry: entry.getValue().entrySet()) {
+ List<IntermediateEntry> intermediateList = partEntry.getValue();
+ if (intermediateList == null || intermediateList.isEmpty()) {
+ continue;
+ }
+ ExecutionBlockId ebId = partEntry.getKey();
+ // EBID + PullHost -> IntermediateEntry
+ // In the case of union partEntry.getKey() return's delegated EBID.
+ // Intermediate entries are merged by real EBID.
+ Map<String, IntermediateEntry> ebMerged = new HashMap<String, IntermediateEntry>();
+
+ for (IntermediateEntry eachIntermediate: intermediateList) {
+ String ebMergedKey = eachIntermediate.getEbId().toString() + eachIntermediate.getPullHost().getPullAddress();
+ IntermediateEntry intermediateEntryPerPullHost = ebMerged.get(ebMergedKey);
+ if (intermediateEntryPerPullHost == null) {
+ intermediateEntryPerPullHost = new IntermediateEntry(-1, -1, partId, eachIntermediate.getPullHost());
+ intermediateEntryPerPullHost.setEbId(eachIntermediate.getEbId());
+ ebMerged.put(ebMergedKey, intermediateEntryPerPullHost);
+ }
+ intermediateEntryPerPullHost.setVolume(intermediateEntryPerPullHost.getVolume() + eachIntermediate.getVolume());
+ }
+
+ List<IntermediateEntry> ebIntermediateEntries = new ArrayList<IntermediateEntry>(ebMerged.values());
+
+ Map<ExecutionBlockId, List<IntermediateEntry>> mergedPartEntries = mergedHashEntries.get(partId);
+ if (mergedPartEntries == null) {
+ mergedPartEntries = new HashMap<ExecutionBlockId, List<IntermediateEntry>>();
+ mergedHashEntries.put(partId, mergedPartEntries);
+ }
+ mergedPartEntries.put(ebId, ebIntermediateEntries);
+ }
+ }
+ return mergedHashEntries;
+ }
+
+ /**
+ * It creates a number of fragments for all partitions.
+ */
+ public static List<Fragment> getFragmentsFromPartitionedTable(FileStorageManager sm,
+ ScanNode scan,
+ TableDesc table) throws IOException {
+ List<Fragment> fragments = Lists.newArrayList();
+ PartitionedTableScanNode partitionsScan = (PartitionedTableScanNode) scan;
+ fragments.addAll(sm.getSplits(
+ scan.getCanonicalName(), table.getMeta(), table.getSchema(), partitionsScan.getInputPaths()));
+ partitionsScan.setInputPaths(null);
+ return fragments;
+ }
+
+ private static void scheduleLeafTasksWithBroadcastTable(TaskSchedulerContext schedulerContext, Stage stage,
+ int baseScanId, Fragment[] fragments) throws IOException {
+ ExecutionBlock execBlock = stage.getBlock();
+ ScanNode[] scans = execBlock.getScanNodes();
+
+ for (int i = 0; i < scans.length; i++) {
+ if (i != baseScanId) {
+ scans[i].setBroadcastTable(true);
+ }
+ }
+
+ // Large table(baseScan)
+ // -> add all fragment to baseFragments
+ // -> each fragment is assigned to a Task by DefaultTaskScheduler.handle()
+ // Broadcast table
+ // all fragments or paths assigned every Large table's scan task.
+ // -> PARTITIONS_SCAN
+ // . add all partition paths to node's inputPaths variable
+ // -> SCAN
+ // . add all fragments to broadcastFragments
+ Collection<Fragment> baseFragments = null;
+ List<Fragment> broadcastFragments = new ArrayList<Fragment>();
+ for (int i = 0; i < scans.length; i++) {
+ ScanNode scan = scans[i];
+ TableDesc desc = stage.getContext().getTableDescMap().get(scan.getCanonicalName());
+ TableMeta meta = desc.getMeta();
+
+ Collection<Fragment> scanFragments;
+ Path[] partitionScanPaths = null;
+ if (scan.getType() == NodeType.PARTITIONS_SCAN) {
+ PartitionedTableScanNode partitionScan = (PartitionedTableScanNode)scan;
+ partitionScanPaths = partitionScan.getInputPaths();
+ // set null to inputPaths in getFragmentsFromPartitionedTable()
+ FileStorageManager storageManager =
+ (FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf());
+ scanFragments = getFragmentsFromPartitionedTable(storageManager, scan, desc);
+ } else {
+ StorageManager storageManager =
+ StorageManager.getStorageManager(stage.getContext().getConf(), desc.getMeta().getStoreType());
+
+ scanFragments = storageManager.getSplits(scan.getCanonicalName(), desc, scan);
+ }
+
+ if (scanFragments != null) {
+ if (i == baseScanId) {
+ baseFragments = scanFragments;
+ } else {
+ if (scan.getType() == NodeType.PARTITIONS_SCAN) {
+ PartitionedTableScanNode partitionScan = (PartitionedTableScanNode)scan;
+ // PhisicalPlanner make PartitionMergeScanExec when table is boradcast table and inputpaths is not empty
+ partitionScan.setInputPaths(partitionScanPaths);
+ } else {
+ broadcastFragments.addAll(scanFragments);
+ }
+ }
+ }
+ }
+
+ if (baseFragments == null) {
+ throw new IOException("No fragments for " + scans[baseScanId].getTableName());
+ }
+
+ Stage.scheduleFragments(stage, baseFragments, broadcastFragments);
+ schedulerContext.setEstimatedTaskNum(baseFragments.size());
+ }
+
+ private static void addJoinShuffle(Stage stage, int partitionId,
+ Map<ExecutionBlockId, List<IntermediateEntry>> grouppedPartitions) {
+ Map<String, List<FetchImpl>> fetches = new HashMap<String, List<FetchImpl>>();
+ for (ExecutionBlock execBlock : stage.getMasterPlan().getChilds(stage.getId())) {
+ if (grouppedPartitions.containsKey(execBlock.getId())) {
+ Collection<FetchImpl> requests = mergeShuffleRequest(partitionId, HASH_SHUFFLE,
+ grouppedPartitions.get(execBlock.getId()));
+ fetches.put(execBlock.getId().toString(), Lists.newArrayList(requests));
+ }
+ }
+
+ if (fetches.isEmpty()) {
+ LOG.info(stage.getId() + "'s " + partitionId + " partition has empty result.");
+ return;
+ }
+ Stage.scheduleFetches(stage, fetches);
+ }
+
+ /**
+ * This method merges the partition request associated with the pullserver's address.
+ * It reduces the number of TCP connections.
+ *
+ * @return key: pullserver's address, value: a list of requests
+ */
+ private static Collection<FetchImpl> mergeShuffleRequest(int partitionId,
+ ShuffleType type,
+ List<IntermediateEntry> partitions) {
+ // ebId + pullhost -> FetchImmpl
+ Map<String, FetchImpl> mergedPartitions = new HashMap<String, FetchImpl>();
+
+ for (IntermediateEntry partition : partitions) {
+ String mergedKey = partition.getEbId().toString() + "," + partition.getPullHost();
+
+ if (mergedPartitions.containsKey(mergedKey)) {
+ FetchImpl fetch = mergedPartitions.get(mergedKey);
+ fetch.addPart(partition.getTaskId(), partition.getAttemptId());
+ } else {
+ // In some cases like union each IntermediateEntry has different EBID.
+ FetchImpl fetch = new FetchImpl(partition.getPullHost(), type, partition.getEbId(), partitionId);
+ fetch.addPart(partition.getTaskId(), partition.getAttemptId());
+ mergedPartitions.put(mergedKey, fetch);
+ }
+ }
+ return mergedPartitions.values();
+ }
+
+ public static void scheduleFragmentsForNonLeafTasks(TaskSchedulerContext schedulerContext,
+ MasterPlan masterPlan, Stage stage, int maxNum)
+ throws IOException {
+ DataChannel channel = masterPlan.getIncomingChannels(stage.getBlock().getId()).get(0);
+ if (channel.getShuffleType() == HASH_SHUFFLE
+ || channel.getShuffleType() == SCATTERED_HASH_SHUFFLE) {
+ scheduleHashShuffledFetches(schedulerContext, masterPlan, stage, channel, maxNum);
+ } else if (channel.getShuffleType() == RANGE_SHUFFLE) {
+ scheduleRangeShuffledFetches(schedulerContext, masterPlan, stage, channel, maxNum);
+ } else {
+ throw new InternalException("Cannot support partition type");
+ }
+ }
+
+ private static TableStats computeChildBlocksStats(QueryMasterTask.QueryMasterTaskContext context, MasterPlan masterPlan,
+ ExecutionBlockId parentBlockId) {
+ List<TableStats> tableStatses = new ArrayList<TableStats>();
+ List<ExecutionBlock> childBlocks = masterPlan.getChilds(parentBlockId);
+ for (ExecutionBlock childBlock : childBlocks) {
+ Stage childStage = context.getStage(childBlock.getId());
+ tableStatses.add(childStage.getResultStats());
+ }
+ return StatisticsUtil.aggregateTableStat(tableStatses);
+ }
+
+ public static void scheduleRangeShuffledFetches(TaskSchedulerContext schedulerContext, MasterPlan masterPlan,
+ Stage stage, DataChannel channel, int maxNum)
+ throws IOException {
+ ExecutionBlock execBlock = stage.getBlock();
+ ScanNode scan = execBlock.getScanNodes()[0];
+ Path tablePath;
+ tablePath = ((FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf()))
+ .getTablePath(scan.getTableName());
+
+ ExecutionBlock sampleChildBlock = masterPlan.getChild(stage.getId(), 0);
+ SortNode sortNode = PlannerUtil.findTopNode(sampleChildBlock.getPlan(), NodeType.SORT);
+ SortSpec [] sortSpecs = sortNode.getSortKeys();
+ Schema sortSchema = new Schema(channel.getShuffleKeys());
+
+ TupleRange[] ranges;
+ int determinedTaskNum;
+
+ // calculate the number of maximum query ranges
+ TableStats totalStat = computeChildBlocksStats(stage.getContext(), masterPlan, stage.getId());
+
+ // If there is an empty table in inner join, it should return zero rows.
+ if (totalStat.getNumBytes() == 0 && totalStat.getColumnStats().size() == 0) {
+ return;
+ }
+ TupleRange mergedRange = TupleUtil.columnStatToRange(sortSpecs, sortSchema, totalStat.getColumnStats(), false);
+
+ if (sortNode.getSortPurpose() == SortPurpose.STORAGE_SPECIFIED) {
+ StoreType storeType = PlannerUtil.getStoreType(masterPlan.getLogicalPlan());
+ CatalogService catalog = stage.getContext().getQueryMasterContext().getWorkerContext().getCatalog();
+ LogicalRootNode rootNode = masterPlan.getLogicalPlan().getRootBlock().getRoot();
+ TableDesc tableDesc = PlannerUtil.getTableDesc(catalog, rootNode.getChild());
+ if (tableDesc == null) {
+ throw new IOException("Can't get table meta data from catalog: " +
+ PlannerUtil.getStoreTableName(masterPlan.getLogicalPlan()));
+ }
+ ranges = StorageManager.getStorageManager(stage.getContext().getConf(), storeType)
+ .getInsertSortRanges(stage.getContext().getQueryContext(), tableDesc,
+ sortNode.getInSchema(), sortSpecs,
+ mergedRange);
+ determinedTaskNum = ranges.length;
+ } else {
+ RangePartitionAlgorithm partitioner = new UniformRangePartition(mergedRange, sortSpecs);
+ BigInteger card = partitioner.getTotalCardinality();
+
+ // if the number of the range cardinality is less than the desired number of tasks,
+ // we set the the number of tasks to the number of range cardinality.
+ if (card.compareTo(BigInteger.valueOf(maxNum)) < 0) {
+ LOG.info(stage.getId() + ", The range cardinality (" + card
+ + ") is less then the desired number of tasks (" + maxNum + ")");
+ determinedTaskNum = card.intValue();
+ } else {
+ determinedTaskNum = maxNum;
+ }
+
+ LOG.info(stage.getId() + ", Try to divide " + mergedRange + " into " + determinedTaskNum +
+ " sub ranges (total units: " + determinedTaskNum + ")");
+ ranges = partitioner.partition(determinedTaskNum);
+ if (ranges == null || ranges.length == 0) {
+ LOG.warn(stage.getId() + " no range infos.");
+ }
+ TupleUtil.setMaxRangeIfNull(sortSpecs, sortSchema, totalStat.getColumnStats(), ranges);
+ if (LOG.isDebugEnabled()) {
+ if (ranges != null) {
+ for (TupleRange eachRange : ranges) {
+ LOG.debug(stage.getId() + " range: " + eachRange.getStart() + " ~ " + eachRange.getEnd());
+ }
+ }
+ }
+ }
+
+ FileFragment dummyFragment = new FileFragment(scan.getTableName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
+ Stage.scheduleFragment(stage, dummyFragment);
+
+ List<FetchImpl> fetches = new ArrayList<FetchImpl>();
+ List<ExecutionBlock> childBlocks = masterPlan.getChilds(stage.getId());
+ for (ExecutionBlock childBlock : childBlocks) {
+ Stage childExecSM = stage.getContext().getStage(childBlock.getId());
+ for (Task qu : childExecSM.getTasks()) {
+ for (IntermediateEntry p : qu.getIntermediateData()) {
+ FetchImpl fetch = new FetchImpl(p.getPullHost(), RANGE_SHUFFLE, childBlock.getId(), 0);
+ fetch.addPart(p.getTaskId(), p.getAttemptId());
+ fetches.add(fetch);
+ }
+ }
+ }
+
+ boolean ascendingFirstKey = sortSpecs[0].isAscending();
+ SortedMap<TupleRange, Collection<FetchImpl>> map;
+ if (ascendingFirstKey) {
+ map = new TreeMap<TupleRange, Collection<FetchImpl>>();
+ } else {
+ map = new TreeMap<TupleRange, Collection<FetchImpl>>(new TupleRange.DescendingTupleRangeComparator());
+ }
+
+ Set<FetchImpl> fetchSet;
+ try {
+ RowStoreUtil.RowStoreEncoder encoder = RowStoreUtil.createEncoder(sortSchema);
+ for (int i = 0; i < ranges.length; i++) {
+ fetchSet = new HashSet<FetchImpl>();
+ for (FetchImpl fetch: fetches) {
+ String rangeParam =
+ TupleUtil.rangeToQuery(ranges[i], ascendingFirstKey ? i == (ranges.length - 1) : i == 0, encoder);
+ FetchImpl copy = null;
+ try {
+ copy = fetch.clone();
+ } catch (CloneNotSupportedException e) {
+ throw new RuntimeException(e);
+ }
+ copy.setRangeParams(rangeParam);
+ fetchSet.add(copy);
+ }
+ map.put(ranges[i], fetchSet);
+ }
+
+ } catch (UnsupportedEncodingException e) {
+ LOG.error(e);
+ }
+
+ scheduleFetchesByRoundRobin(stage, map, scan.getTableName(), determinedTaskNum);
+
+ schedulerContext.setEstimatedTaskNum(determinedTaskNum);
+ }
+
+ public static void scheduleFetchesByRoundRobin(Stage stage, Map<?, Collection<FetchImpl>> partitions,
+ String tableName, int num) {
+ int i;
+ Map<String, List<FetchImpl>>[] fetchesArray = new Map[num];
+ for (i = 0; i < num; i++) {
+ fetchesArray[i] = new HashMap<String, List<FetchImpl>>();
+ }
+ i = 0;
+ for (Entry<?, Collection<FetchImpl>> entry : partitions.entrySet()) {
+ Collection<FetchImpl> value = entry.getValue();
+ TUtil.putCollectionToNestedList(fetchesArray[i++], tableName, value);
+ if (i == num) i = 0;
+ }
+ for (Map<String, List<FetchImpl>> eachFetches : fetchesArray) {
+ Stage.scheduleFetches(stage, eachFetches);
+ }
+ }
+
+ @VisibleForTesting
+ public static class FetchGroupMeta {
+ long totalVolume;
+ List<FetchImpl> fetchUrls;
+
+ public FetchGroupMeta(long volume, FetchImpl fetchUrls) {
+ this.totalVolume = volume;
+ this.fetchUrls = Lists.newArrayList(fetchUrls);
+ }
+
+ public FetchGroupMeta addFetche(FetchImpl fetches) {
+ this.fetchUrls.add(fetches);
+ return this;
+ }
+
+ public void increaseVolume(long volume) {
+ this.totalVolume += volume;
+ }
+
+ public long getVolume() {
+ return totalVolume;
+ }
+
+ }
+
+ public static void scheduleHashShuffledFetches(TaskSchedulerContext schedulerContext, MasterPlan masterPlan,
+ Stage stage, DataChannel channel,
+ int maxNum) throws IOException {
+ ExecutionBlock execBlock = stage.getBlock();
+ ScanNode scan = execBlock.getScanNodes()[0];
+ Path tablePath;
+ tablePath = ((FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf()))
+ .getTablePath(scan.getTableName());
+
+ Fragment frag = new FileFragment(scan.getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
+ List<Fragment> fragments = new ArrayList<Fragment>();
+ fragments.add(frag);
+ Stage.scheduleFragments(stage, fragments);
+
+ Map<Integer, FetchGroupMeta> finalFetches = new HashMap<Integer, FetchGroupMeta>();
+ Map<ExecutionBlockId, List<IntermediateEntry>> intermediates = new HashMap<ExecutionBlockId,
+ List<IntermediateEntry>>();
+
+ for (ExecutionBlock block : masterPlan.getChilds(execBlock)) {
+ List<IntermediateEntry> partitions = new ArrayList<IntermediateEntry>();
+ partitions.addAll(stage.getContext().getStage(block.getId()).getHashShuffleIntermediateEntries());
+
+ // In scattered hash shuffle, Collecting each IntermediateEntry
+ if (channel.getShuffleType() == SCATTERED_HASH_SHUFFLE) {
+ if (intermediates.containsKey(block.getId())) {
+ intermediates.get(block.getId()).addAll(partitions);
+ } else {
+ intermediates.put(block.getId(), partitions);
+ }
+ }
+
+ // make FetchImpl per PullServer, PartId
+ Map<Integer, List<IntermediateEntry>> hashed = hashByKey(partitions);
+ for (Entry<Integer, List<IntermediateEntry>> interm : hashed.entrySet()) {
+ Map<Task.PullHost, List<IntermediateEntry>> hashedByHost = hashByHost(interm.getValue());
+ for (Entry<Task.PullHost, List<IntermediateEntry>> e : hashedByHost.entrySet()) {
+
+ FetchImpl fetch = new FetchImpl(e.getKey(), channel.getShuffleType(),
+ block.getId(), interm.getKey(), e.getValue());
+
+ long volumeSum = 0;
+ for (IntermediateEntry ie : e.getValue()) {
+ volumeSum += ie.getVolume();
+ }
+
+ if (finalFetches.containsKey(interm.getKey())) {
+ finalFetches.get(interm.getKey()).addFetche(fetch).increaseVolume(volumeSum);
+ } else {
+ finalFetches.put(interm.getKey(), new FetchGroupMeta(volumeSum, fetch));
+ }
+ }
+ }
+ }
+
+ int groupingColumns = 0;
+ LogicalNode[] groupbyNodes = PlannerUtil.findAllNodes(stage.getBlock().getPlan(),
+ new NodeType[]{NodeType.GROUP_BY, NodeType.DISTINCT_GROUP_BY});
+ if (groupbyNodes != null && groupbyNodes.length > 0) {
+ LogicalNode bottomNode = groupbyNodes[0];
+ if (bottomNode.getType() == NodeType.GROUP_BY) {
+ groupingColumns = ((GroupbyNode)bottomNode).getGroupingColumns().length;
+ } else if (bottomNode.getType() == NodeType.DISTINCT_GROUP_BY) {
+ DistinctGroupbyNode distinctNode = PlannerUtil.findMostBottomNode(stage.getBlock().getPlan(), NodeType.DISTINCT_GROUP_BY);
+ if (distinctNode == null) {
+ LOG.warn(stage.getId() + ", Can't find current DistinctGroupbyNode");
+ distinctNode = (DistinctGroupbyNode)bottomNode;
+ }
+ groupingColumns = distinctNode.getGroupingColumns().length;
+
+ Enforcer enforcer = execBlock.getEnforcer();
+ EnforceProperty property = PhysicalPlannerImpl.getAlgorithmEnforceProperty(enforcer, distinctNode);
+ if (property != null) {
+ if (property.getDistinct().getIsMultipleAggregation()) {
+ MultipleAggregationStage mulAggStage = property.getDistinct().getMultipleAggregationStage();
+ if (mulAggStage != MultipleAggregationStage.THRID_STAGE) {
+ groupingColumns = distinctNode.getOutSchema().size();
+ }
+ }
+ }
+ }
+ }
+ // get a proper number of tasks
+ int determinedTaskNum = Math.min(maxNum, finalFetches.size());
+ LOG.info(stage.getId() + ", ScheduleHashShuffledFetches - Max num=" + maxNum + ", finalFetchURI=" + finalFetches.size());
+
+ if (groupingColumns == 0) {
+ determinedTaskNum = 1;
+ LOG.info(stage.getId() + ", No Grouping Column - determinedTaskNum is set to 1");
+ } else {
+ TableStats totalStat = computeChildBlocksStats(stage.getContext(), masterPlan, stage.getId());
+ if (totalStat.getNumRows() == 0) {
+ determinedTaskNum = 1;
+ }
+ }
+
+ // set the proper number of tasks to the estimated task num
+ if (channel.getShuffleType() == SCATTERED_HASH_SHUFFLE) {
+ scheduleScatteredHashShuffleFetches(schedulerContext, stage, intermediates,
+ scan.getTableName());
+ } else {
+ schedulerContext.setEstimatedTaskNum(determinedTaskNum);
+ // divide fetch uris into the the proper number of tasks according to volumes
+ scheduleFetchesByEvenDistributedVolumes(stage, finalFetches, scan.getTableName(), determinedTaskNum);
+ LOG.info(stage.getId() + ", DeterminedTaskNum : " + determinedTaskNum);
+ }
+ }
+
+ public static Pair<Long [], Map<String, List<FetchImpl>>[]> makeEvenDistributedFetchImpl(
+ Map<Integer, FetchGroupMeta> partitions, String tableName, int num) {
+
+ // Sort fetchGroupMeta in a descending order of data volumes.
+ List<FetchGroupMeta> fetchGroupMetaList = Lists.newArrayList(partitions.values());
+ Collections.sort(fetchGroupMetaList, new Comparator<FetchGroupMeta>() {
+ @Override
+ public int compare(FetchGroupMeta o1, FetchGroupMeta o2) {
+ return o1.getVolume() < o2.getVolume() ? 1 : (o1.getVolume() > o2.getVolume() ? -1 : 0);
+ }
+ });
+
+ // Initialize containers
+ Map<String, List<FetchImpl>>[] fetchesArray = new Map[num];
+ Long [] assignedVolumes = new Long[num];
+ // initialization
+ for (int i = 0; i < num; i++) {
+ fetchesArray[i] = new HashMap<String, List<FetchImpl>>();
+ assignedVolumes[i] = 0l;
+ }
+
+ // This algorithm assignes bigger first manner by using a sorted iterator. It is a kind of greedy manner.
+ // Its complexity is O(n). Since FetchGroup can be more than tens of thousands, we should consider its complexity.
+ // In terms of this point, it will show reasonable performance and results. even though it is not an optimal
+ // algorithm.
+ Iterator<FetchGroupMeta> iterator = fetchGroupMetaList.iterator();
+
+ int p = 0;
+ while(iterator.hasNext()) {
+ while (p < num && iterator.hasNext()) {
+ FetchGroupMeta fetchGroupMeta = iterator.next();
+ assignedVolumes[p] += fetchGroupMeta.getVolume();
+
+ TUtil.putCollectionToNestedList(fetchesArray[p], tableName, fetchGroupMeta.fetchUrls);
+ p++;
+ }
+
+ p = num - 1;
+ while (p > 0 && iterator.hasNext()) {
+ FetchGroupMeta fetchGroupMeta = iterator.next();
+ assignedVolumes[p] += fetchGroupMeta.getVolume();
+ TUtil.putCollectionToNestedList(fetchesArray[p], tableName, fetchGroupMeta.fetchUrls);
+
+ // While the current one is smaller than next one, it adds additional fetches to current one.
+ while(iterator.hasNext() && assignedVolumes[p - 1] > assignedVolumes[p]) {
+ FetchGroupMeta additionalFetchGroup = iterator.next();
+ assignedVolumes[p] += additionalFetchGroup.getVolume();
+ TUtil.putCollectionToNestedList(fetchesArray[p], tableName, additionalFetchGroup.fetchUrls);
+ }
+
+ p--;
+ }
+ }
+
+ return new Pair<Long[], Map<String, List<FetchImpl>>[]>(assignedVolumes, fetchesArray);
+ }
+
+ public static void scheduleFetchesByEvenDistributedVolumes(Stage stage, Map<Integer, FetchGroupMeta> partitions,
+ String tableName, int num) {
+ Map<String, List<FetchImpl>>[] fetchsArray = makeEvenDistributedFetchImpl(partitions, tableName, num).getSecond();
+ // Schedule FetchImpls
+ for (Map<String, List<FetchImpl>> eachFetches : fetchsArray) {
+ Stage.scheduleFetches(stage, eachFetches);
+ }
+ }
+
+ // Scattered hash shuffle hashes the key columns and groups the hash keys associated with
+ // the same hash key. Then, if the volume of a group is larger
+ // than $DIST_QUERY_TABLE_PARTITION_VOLUME, it divides the group into more than two sub groups
+ // according to $DIST_QUERY_TABLE_PARTITION_VOLUME (default size = 256MB).
+ // As a result, each group size always becomes the less than or equal
+ // to $DIST_QUERY_TABLE_PARTITION_VOLUME. Finally, each subgroup is assigned to a query unit.
+ // It is usually used for writing partitioned tables.
+ public static void scheduleScatteredHashShuffleFetches(TaskSchedulerContext schedulerContext,
+ Stage stage, Map<ExecutionBlockId, List<IntermediateEntry>> intermediates,
+ String tableName) {
+ long splitVolume = StorageUnit.MB *
+ stage.getMasterPlan().getContext().getLong(SessionVars.TABLE_PARTITION_PER_SHUFFLE_SIZE);
+ long pageSize = StorageUnit.MB *
+ stage.getContext().getConf().getIntVar(ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME); // in bytes
+ if (pageSize >= splitVolume) {
+ throw new RuntimeException("tajo.dist-query.table-partition.task-volume-mb should be great than " +
+ "tajo.shuffle.hash.appender.page.volumn-mb");
+ }
+ List<List<FetchImpl>> fetches = new ArrayList<List<FetchImpl>>();
+
+ long totalIntermediateSize = 0L;
+ for (Entry<ExecutionBlockId, List<IntermediateEntry>> listEntry : intermediates.entrySet()) {
+ // merge by PartitionId
+ Map<Integer, List<IntermediateEntry>> partitionIntermMap = new HashMap<Integer, List<IntermediateEntry>>();
+ for (IntermediateEntry eachInterm: listEntry.getValue()) {
+ totalIntermediateSize += eachInterm.getVolume();
+ int partId = eachInterm.getPartId();
+ List<IntermediateEntry> partitionInterms = partitionIntermMap.get(partId);
+ if (partitionInterms == null) {
+ partitionInterms = TUtil.newList(eachInterm);
+ partitionIntermMap.put(partId, partitionInterms);
+ } else {
+ partitionInterms.add(eachInterm);
+ }
+ }
+
+ // Grouping or splitting to fit $DIST_QUERY_TABLE_PARTITION_VOLUME size
+ for (List<IntermediateEntry> partitionEntries : partitionIntermMap.values()) {
+ List<List<FetchImpl>> eachFetches = splitOrMergeIntermediates(listEntry.getKey(), partitionEntries,
+ splitVolume, pageSize);
+ if (eachFetches != null && !eachFetches.isEmpty()) {
+ fetches.addAll(eachFetches);
+ }
+ }
+ }
+
+ schedulerContext.setEstimatedTaskNum(fetches.size());
+
+ int i = 0;
+ Map<String, List<FetchImpl>>[] fetchesArray = new Map[fetches.size()];
+ for(List<FetchImpl> entry : fetches) {
+ fetchesArray[i] = new HashMap<String, List<FetchImpl>>();
+ fetchesArray[i].put(tableName, entry);
+
+ Stage.scheduleFetches(stage, fetchesArray[i]);
+ i++;
+ }
+
+ LOG.info(stage.getId()
+ + ", ShuffleType:" + SCATTERED_HASH_SHUFFLE.name()
+ + ", Intermediate Size: " + totalIntermediateSize
+ + ", splitSize: " + splitVolume
+ + ", DeterminedTaskNum: " + fetches.size());
+ }
+
+ /**
+ * If a IntermediateEntry is large than splitVolume, List<FetchImpl> has single element.
+ * @param ebId
+ * @param entries
+ * @param splitVolume
+ * @return
+ */
+ public static List<List<FetchImpl>> splitOrMergeIntermediates(
+ ExecutionBlockId ebId, List<IntermediateEntry> entries, long splitVolume, long pageSize) {
+ // Each List<FetchImpl> has splitVolume size.
+ List<List<FetchImpl>> fetches = new ArrayList<List<FetchImpl>>();
+
+ Iterator<IntermediateEntry> iter = entries.iterator();
+ if (!iter.hasNext()) {
+ return null;
+ }
+ List<FetchImpl> fetchListForSingleTask = new ArrayList<FetchImpl>();
+ long fetchListVolume = 0;
+
+ while (iter.hasNext()) {
+ IntermediateEntry currentInterm = iter.next();
+
+ long firstSplitVolume = splitVolume - fetchListVolume;
+ if (firstSplitVolume < pageSize) {
+ firstSplitVolume = splitVolume;
+ }
+
+ //Each Pair object in the splits variable is assigned to the next ExectionBlock's task.
+ //The first long value is a offset of the intermediate file and the second long value is length.
+ List<Pair<Long, Long>> splits = currentInterm.split(firstSplitVolume, splitVolume);
+ if (splits == null || splits.isEmpty()) {
+ break;
+ }
+
+ for (Pair<Long, Long> eachSplit: splits) {
+ if (fetchListVolume > 0 && fetchListVolume + eachSplit.getSecond() >= splitVolume) {
+ if (!fetchListForSingleTask.isEmpty()) {
+ fetches.add(fetchListForSingleTask);
+ }
+ fetchListForSingleTask = new ArrayList<FetchImpl>();
+ fetchListVolume = 0;
+ }
+ FetchImpl fetch = new FetchImpl(currentInterm.getPullHost(), SCATTERED_HASH_SHUFFLE,
+ ebId, currentInterm.getPartId(), TUtil.newList(currentInterm));
+ fetch.setOffset(eachSplit.getFirst());
+ fetch.setLength(eachSplit.getSecond());
+ fetchListForSingleTask.add(fetch);
+ fetchListVolume += eachSplit.getSecond();
+ }
+ }
+ if (!fetchListForSingleTask.isEmpty()) {
+ fetches.add(fetchListForSingleTask);
+ }
+ return fetches;
+ }
+
+ public static List<URI> createFetchURL(FetchImpl fetch, boolean includeParts) {
+ String scheme = "http://";
+
+ StringBuilder urlPrefix = new StringBuilder(scheme);
+ urlPrefix.append(fetch.getPullHost().getHost()).append(":").append(fetch.getPullHost().getPort()).append("/?")
+ .append("qid=").append(fetch.getExecutionBlockId().getQueryId().toString())
+ .append("&sid=").append(fetch.getExecutionBlockId().getId())
+ .append("&p=").append(fetch.getPartitionId())
+ .append("&type=");
+ if (fetch.getType() == HASH_SHUFFLE) {
+ urlPrefix.append("h");
+ } else if (fetch.getType() == RANGE_SHUFFLE) {
+ urlPrefix.append("r").append("&").append(fetch.getRangeParams());
+ } else if (fetch.getType() == SCATTERED_HASH_SHUFFLE) {
+ urlPrefix.append("s");
+ }
+
+ if (fetch.getLength() >= 0) {
+ urlPrefix.append("&offset=").append(fetch.getOffset()).append("&length=").append(fetch.getLength());
+ }
+
+ List<URI> fetchURLs = new ArrayList<URI>();
+ if(includeParts) {
+ if (fetch.getType() == HASH_SHUFFLE || fetch.getType() == SCATTERED_HASH_SHUFFLE) {
+ fetchURLs.add(URI.create(urlPrefix.toString()));
+ } else {
+ // If the get request is longer than 2000 characters,
+ // the long request uri may cause HTTP Status Code - 414 Request-URI Too Long.
+ // Refer to http://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html#sec10.4.15
+ // The below code transforms a long request to multiple requests.
+ List<String> taskIdsParams = new ArrayList<String>();
+ StringBuilder taskIdListBuilder = new StringBuilder();
+ List<Integer> taskIds = fetch.getTaskIds();
+ List<Integer> attemptIds = fetch.getAttemptIds();
+ boolean first = true;
+
+ for (int i = 0; i < taskIds.size(); i++) {
+ StringBuilder taskAttemptId = new StringBuilder();
+
+ if (!first) { // when comma is added?
+ taskAttemptId.append(",");
+ } else {
+ first = false;
+ }
+
+ int taskId = taskIds.get(i);
+ if (taskId < 0) {
+ // In the case of hash shuffle each partition has single shuffle file per worker.
+ // TODO If file is large, consider multiple fetching(shuffle file can be split)
+ continue;
+ }
+ int attemptId = attemptIds.get(i);
+ taskAttemptId.append(taskId).append("_").append(attemptId);
+
+ if (taskIdListBuilder.length() + taskAttemptId.length()
+ > HTTP_REQUEST_MAXIMUM_LENGTH) {
+ taskIdsParams.add(taskIdListBuilder.toString());
+ taskIdListBuilder = new StringBuilder(taskId + "_" + attemptId);
+ } else {
+ taskIdListBuilder.append(taskAttemptId);
+ }
+ }
+ // if the url params remain
+ if (taskIdListBuilder.length() > 0) {
+ taskIdsParams.add(taskIdListBuilder.toString());
+ }
+ urlPrefix.append("&ta=");
+ for (String param : taskIdsParams) {
+ fetchURLs.add(URI.create(urlPrefix + param));
+ }
+ }
+ } else {
+ fetchURLs.add(URI.create(urlPrefix.toString()));
+ }
+
+ return fetchURLs;
+ }
+
+ public static Map<Integer, List<IntermediateEntry>> hashByKey(List<IntermediateEntry> entries) {
+ Map<Integer, List<IntermediateEntry>> hashed = new HashMap<Integer, List<IntermediateEntry>>();
+ for (IntermediateEntry entry : entries) {
+ if (hashed.containsKey(entry.getPartId())) {
+ hashed.get(entry.getPartId()).add(entry);
+ } else {
+ hashed.put(entry.getPartId(), TUtil.newList(entry));
+ }
+ }
+
+ return hashed;
+ }
+
+ public static Map<Task.PullHost, List<IntermediateEntry>> hashByHost(List<IntermediateEntry> entries) {
+ Map<Task.PullHost, List<IntermediateEntry>> hashed = new HashMap<Task.PullHost, List<IntermediateEntry>>();
+
+ Task.PullHost host;
+ for (IntermediateEntry entry : entries) {
+ host = entry.getPullHost();
+ if (hashed.containsKey(host)) {
+ hashed.get(host).add(entry);
+ } else {
+ hashed.put(host, TUtil.newList(entry));
+ }
+ }
+
+ return hashed;
+ }
+
+ public static Stage setShuffleOutputNumForTwoPhase(Stage stage, final int desiredNum, DataChannel channel) {
+ ExecutionBlock execBlock = stage.getBlock();
+ Column[] keys;
+ // if the next query is join,
+ // set the partition number for the current logicalUnit
+ // TODO: the union handling is required when a join has unions as its child
+ MasterPlan masterPlan = stage.getMasterPlan();
+ keys = channel.getShuffleKeys();
+ if (!masterPlan.isRoot(stage.getBlock()) ) {
+ ExecutionBlock parentBlock = masterPlan.getParent(stage.getBlock());
+ if (parentBlock.getPlan().getType() == NodeType.JOIN) {
+ channel.setShuffleOutputNum(desiredNum);
+ }
+ }
+
+ // set the partition number for group by and sort
+ if (channel.getShuffleType() == HASH_SHUFFLE) {
+ if (execBlock.getPlan().getType() == NodeType.GROUP_BY ||
+ execBlock.getPlan().getType() == NodeType.DISTINCT_GROUP_BY) {
+ keys = channel.getShuffleKeys();
+ }
+ } else if (channel.getShuffleType() == RANGE_SHUFFLE) {
+ if (execBlock.getPlan().getType() == NodeType.SORT) {
+ SortNode sort = (SortNode) execBlock.getPlan();
+ keys = new Column[sort.getSortKeys().length];
+ for (int i = 0; i < keys.length; i++) {
+ keys[i] = sort.getSortKeys()[i].getSortKey();
+ }
+ }
+ }
+ if (keys != null) {
+ if (keys.length == 0) {
+ channel.setShuffleKeys(new Column[]{});
+ channel.setShuffleOutputNum(1);
+ } else {
+ channel.setShuffleKeys(keys);
+ channel.setShuffleOutputNum(desiredNum);
+ }
+ }
+ return stage;
+ }
+}
[12/16] tajo git commit: TAJO-1288: Refactoring
org.apache.tajo.master package.
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java
new file mode 100644
index 0000000..f645dc5
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java
@@ -0,0 +1,616 @@
+/**
+ * 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.tajo.master.exec;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Stack;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TaskAttemptId;
+import org.apache.tajo.TaskId;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.DatabaseProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.IndexProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableDescriptorProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableOptionProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TablePartitionProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TableStatsProto;
+import org.apache.tajo.catalog.proto.CatalogProtos.TablespaceProto;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.engine.codegen.CompilationError;
+import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
+import org.apache.tajo.engine.planner.Projector;
+import org.apache.tajo.engine.planner.global.ExecutionBlock;
+import org.apache.tajo.engine.planner.global.ExecutionBlockCursor;
+import org.apache.tajo.engine.planner.global.GlobalPlanner;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.planner.physical.PhysicalExec;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.master.TajoMaster.MasterContext;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.PlanningException;
+import org.apache.tajo.plan.expr.EvalNode;
+import org.apache.tajo.plan.logical.IndexScanNode;
+import org.apache.tajo.plan.logical.LogicalNode;
+import org.apache.tajo.plan.logical.ScanNode;
+import org.apache.tajo.storage.RowStoreUtil;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.VTuple;
+import org.apache.tajo.storage.RowStoreUtil.RowStoreEncoder;
+import org.apache.tajo.util.KeyValueSet;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import com.google.protobuf.ByteString;
+
+public class NonForwardQueryResultSystemScanner implements NonForwardQueryResultScanner {
+
+ private final Log LOG = LogFactory.getLog(getClass());
+
+ private MasterContext masterContext;
+ private LogicalPlan logicalPlan;
+ private final QueryId queryId;
+ private final String sessionId;
+ private TaskAttemptContext taskContext;
+ private int currentRow;
+ private long maxRow;
+ private TableDesc tableDesc;
+ private Schema outSchema;
+ private RowStoreEncoder encoder;
+ private PhysicalExec physicalExec;
+
+ public NonForwardQueryResultSystemScanner(MasterContext context, LogicalPlan plan, QueryId queryId,
+ String sessionId, int maxRow) {
+ masterContext = context;
+ logicalPlan = plan;
+ this.queryId = queryId;
+ this.sessionId = sessionId;
+ this.maxRow = maxRow;
+
+ }
+
+ @Override
+ public void init() throws IOException {
+ QueryContext queryContext = new QueryContext(masterContext.getConf());
+ currentRow = 0;
+
+ MasterPlan masterPlan = new MasterPlan(queryId, queryContext, logicalPlan);
+ GlobalPlanner globalPlanner = new GlobalPlanner(masterContext.getConf(), masterContext.getCatalog());
+ try {
+ globalPlanner.build(masterPlan);
+ } catch (PlanningException e) {
+ throw new RuntimeException(e);
+ }
+
+ ExecutionBlockCursor cursor = new ExecutionBlockCursor(masterPlan);
+ ExecutionBlock leafBlock = null;
+ while (cursor.hasNext()) {
+ ExecutionBlock block = cursor.nextBlock();
+ if (masterPlan.isLeaf(block)) {
+ leafBlock = block;
+ break;
+ }
+ }
+
+ taskContext = new TaskAttemptContext(queryContext, null,
+ new TaskAttemptId(new TaskId(leafBlock.getId(), 0), 0),
+ null, null);
+ physicalExec = new SimplePhysicalPlannerImpl(masterContext.getConf())
+ .createPlan(taskContext, leafBlock.getPlan());
+
+ tableDesc = new TableDesc("table_"+System.currentTimeMillis(), physicalExec.getSchema(),
+ new TableMeta(StoreType.SYSTEM, new KeyValueSet()), null);
+ outSchema = physicalExec.getSchema();
+ encoder = RowStoreUtil.createEncoder(getLogicalSchema());
+
+ physicalExec.init();
+ }
+
+ @Override
+ public void close() throws Exception {
+ tableDesc = null;
+ outSchema = null;
+ encoder = null;
+ if (physicalExec != null) {
+ try {
+ physicalExec.close();
+ } catch (Exception ignored) {}
+ }
+ physicalExec = null;
+ currentRow = -1;
+ }
+
+ private List<Tuple> getTablespaces(Schema outSchema) {
+ List<TablespaceProto> tablespaces = masterContext.getCatalog().getAllTablespaces();
+ List<Tuple> tuples = new ArrayList<Tuple>(tablespaces.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+
+ for (TablespaceProto tablespace: tablespaces) {
+ aTuple = new VTuple(outSchema.size());
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column column = columns.get(fieldId);
+ if ("space_id".equalsIgnoreCase(column.getSimpleName())) {
+ if (tablespace.hasId()) {
+ aTuple.put(fieldId, DatumFactory.createInt4(tablespace.getId()));
+ } else {
+ aTuple.put(fieldId, DatumFactory.createNullDatum());
+ }
+ } else if ("space_name".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(tablespace.getSpaceName()));
+ } else if ("space_handler".equalsIgnoreCase(column.getSimpleName())) {
+ if (tablespace.hasHandler()) {
+ aTuple.put(fieldId, DatumFactory.createText(tablespace.getHandler()));
+ } else {
+ aTuple.put(fieldId, DatumFactory.createNullDatum());
+ }
+ } else if ("space_uri".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(tablespace.getUri()));
+ }
+ }
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
+
+ private List<Tuple> getDatabases(Schema outSchema) {
+ List<DatabaseProto> databases = masterContext.getCatalog().getAllDatabases();
+ List<Tuple> tuples = new ArrayList<Tuple>(databases.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+
+ for (DatabaseProto database: databases) {
+ aTuple = new VTuple(outSchema.size());
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column column = columns.get(fieldId);
+ if ("db_id".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(database.getId()));
+ } else if ("db_name".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(database.getName()));
+ } else if ("space_id".equalsIgnoreCase(column.getSimpleName())) {
+ if (database.hasSpaceId()) {
+ aTuple.put(fieldId, DatumFactory.createInt4(database.getSpaceId()));
+ } else {
+ aTuple.put(fieldId, DatumFactory.createNullDatum());
+ }
+ }
+ }
+
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
+
+ private List<Tuple> getTables(Schema outSchema) {
+ List<TableDescriptorProto> tables = masterContext.getCatalog().getAllTables();
+ List<Tuple> tuples = new ArrayList<Tuple>(tables.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+
+ for (TableDescriptorProto table: tables) {
+ aTuple = new VTuple(outSchema.size());
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column column = columns.get(fieldId);
+ if ("tid".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(table.getTid()));
+ } else if ("db_id".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(table.getDbId()));
+ } else if ("table_name".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(table.getName()));
+ } else if ("table_type".equalsIgnoreCase(column.getSimpleName())) {
+ if (table.hasTableType()) {
+ aTuple.put(fieldId, DatumFactory.createText(table.getTableType()));
+ } else {
+ aTuple.put(fieldId, DatumFactory.createNullDatum());
+ }
+ } else if ("path".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(table.getPath()));
+ } else if ("store_type".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(table.getStoreType()));
+ }
+ }
+
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
+
+ private List<Tuple> getColumns(Schema outSchema) {
+ List<ColumnProto> columnsList = masterContext.getCatalog().getAllColumns();
+ List<Tuple> tuples = new ArrayList<Tuple>(columnsList.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+ int columnId = 1, prevtid = -1, tid = 0;
+
+ for (ColumnProto column: columnsList) {
+ aTuple = new VTuple(outSchema.size());
+
+ tid = column.getTid();
+ if (prevtid != tid) {
+ columnId = 1;
+ prevtid = tid;
+ }
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column colObj = columns.get(fieldId);
+
+ if ("tid".equalsIgnoreCase(colObj.getSimpleName())) {
+ if (column.hasTid()) {
+ aTuple.put(fieldId, DatumFactory.createInt4(tid));
+ } else {
+ aTuple.put(fieldId, DatumFactory.createNullDatum());
+ }
+ } else if ("column_name".equalsIgnoreCase(colObj.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(column.getName()));
+ } else if ("ordinal_position".equalsIgnoreCase(colObj.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(columnId));
+ } else if ("data_type".equalsIgnoreCase(colObj.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(column.getDataType().getType().toString()));
+ } else if ("type_length".equalsIgnoreCase(colObj.getSimpleName())) {
+ DataType dataType = column.getDataType();
+ if (dataType.hasLength()) {
+ aTuple.put(fieldId, DatumFactory.createInt4(dataType.getLength()));
+ } else {
+ aTuple.put(fieldId, DatumFactory.createNullDatum());
+ }
+ }
+ }
+
+ columnId++;
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
+
+ private List<Tuple> getIndexes(Schema outSchema) {
+ List<IndexProto> indexList = masterContext.getCatalog().getAllIndexes();
+ List<Tuple> tuples = new ArrayList<Tuple>(indexList.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+
+ for (IndexProto index: indexList) {
+ aTuple = new VTuple(outSchema.size());
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column column = columns.get(fieldId);
+
+ if ("db_id".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(index.getDbId()));
+ } else if ("tid".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(index.getTId()));
+ } else if ("index_name".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(index.getIndexName()));
+ } else if ("column_name".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(index.getColumnName()));
+ } else if ("data_type".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(index.getDataType()));
+ } else if ("index_type".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(index.getIndexType()));
+ } else if ("is_unique".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createBool(index.getIsUnique()));
+ } else if ("is_clustered".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createBool(index.getIsClustered()));
+ } else if ("is_ascending".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createBool(index.getIsAscending()));
+ }
+ }
+
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
+
+ private List<Tuple> getAllTableOptions(Schema outSchema) {
+ List<TableOptionProto> optionList = masterContext.getCatalog().getAllTableOptions();
+ List<Tuple> tuples = new ArrayList<Tuple>(optionList.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+
+ for (TableOptionProto option: optionList) {
+ aTuple = new VTuple(outSchema.size());
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column column = columns.get(fieldId);
+
+ if ("tid".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(option.getTid()));
+ } else if ("key_".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(option.getKeyval().getKey()));
+ } else if ("value_".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(option.getKeyval().getValue()));
+ }
+ }
+
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
+
+ private List<Tuple> getAllTableStats(Schema outSchema) {
+ List<TableStatsProto> statList = masterContext.getCatalog().getAllTableStats();
+ List<Tuple> tuples = new ArrayList<Tuple>(statList.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+
+ for (TableStatsProto stat: statList) {
+ aTuple = new VTuple(outSchema.size());
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column column = columns.get(fieldId);
+
+ if ("tid".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(stat.getTid()));
+ } else if ("num_rows".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt8(stat.getNumRows()));
+ } else if ("num_bytes".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt8(stat.getNumBytes()));
+ }
+ }
+
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
+
+ private List<Tuple> getAllPartitions(Schema outSchema) {
+ List<TablePartitionProto> partitionList = masterContext.getCatalog().getAllPartitions();
+ List<Tuple> tuples = new ArrayList<Tuple>(partitionList.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+
+ for (TablePartitionProto partition: partitionList) {
+ aTuple = new VTuple(outSchema.size());
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column column = columns.get(fieldId);
+
+ if ("pid".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(partition.getPid()));
+ } else if ("tid".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(partition.getTid()));
+ } else if ("partition_name".equalsIgnoreCase(column.getSimpleName())) {
+ if (partition.hasPartitionName()) {
+ aTuple.put(fieldId, DatumFactory.createText(partition.getPartitionName()));
+ } else {
+ aTuple.put(fieldId, DatumFactory.createNullDatum());
+ }
+ } else if ("ordinal_position".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(partition.getOrdinalPosition()));
+ } else if ("path".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(partition.getPath()));
+ }
+ }
+
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
+
+ private List<Tuple> fetchSystemTable(TableDesc tableDesc, Schema inSchema) {
+ List<Tuple> tuples = null;
+ String tableName = CatalogUtil.extractSimpleName(tableDesc.getName());
+
+ if ("tablespace".equalsIgnoreCase(tableName)) {
+ tuples = getTablespaces(inSchema);
+ } else if ("databases".equalsIgnoreCase(tableName)) {
+ tuples = getDatabases(inSchema);
+ } else if ("tables".equalsIgnoreCase(tableName)) {
+ tuples = getTables(inSchema);
+ } else if ("columns".equalsIgnoreCase(tableName)) {
+ tuples = getColumns(inSchema);
+ } else if ("indexes".equalsIgnoreCase(tableName)) {
+ tuples = getIndexes(inSchema);
+ } else if ("table_options".equalsIgnoreCase(tableName)) {
+ tuples = getAllTableOptions(inSchema);
+ } else if ("table_stats".equalsIgnoreCase(tableName)) {
+ tuples = getAllTableStats(inSchema);
+ } else if ("partitions".equalsIgnoreCase(tableName)) {
+ tuples = getAllPartitions(inSchema);
+ }
+
+ return tuples;
+ }
+
+ @Override
+ public List<ByteString> getNextRows(int fetchRowNum) throws IOException {
+ List<ByteString> rows = new ArrayList<ByteString>();
+ int startRow = currentRow;
+ int endRow = startRow + fetchRowNum;
+
+ if (physicalExec == null) {
+ return rows;
+ }
+
+ while (currentRow < endRow) {
+ Tuple currentTuple = physicalExec.next();
+
+ if (currentTuple == null) {
+ physicalExec.close();
+ physicalExec = null;
+ break;
+ }
+
+ currentRow++;
+ rows.add(ByteString.copyFrom(encoder.toBytes(currentTuple)));
+
+ if (currentRow >= maxRow) {
+ physicalExec.close();
+ physicalExec = null;
+ break;
+ }
+ }
+
+ return rows;
+ }
+
+ @Override
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ @Override
+ public String getSessionId() {
+ return sessionId;
+ }
+
+ @Override
+ public TableDesc getTableDesc() {
+ return tableDesc;
+ }
+
+ @Override
+ public Schema getLogicalSchema() {
+ return outSchema;
+ }
+
+ class SimplePhysicalPlannerImpl extends PhysicalPlannerImpl {
+
+ public SimplePhysicalPlannerImpl(TajoConf conf) {
+ super(conf);
+ }
+
+ @Override
+ public PhysicalExec createScanPlan(TaskAttemptContext ctx, ScanNode scanNode, Stack<LogicalNode> node)
+ throws IOException {
+ return new SystemPhysicalExec(ctx, scanNode);
+ }
+
+ @Override
+ public PhysicalExec createIndexScanExec(TaskAttemptContext ctx, IndexScanNode annotation) throws IOException {
+ return new SystemPhysicalExec(ctx, annotation);
+ }
+ }
+
+ class SystemPhysicalExec extends PhysicalExec {
+
+ private ScanNode scanNode;
+ private EvalNode qual;
+ private Projector projector;
+ private TableStats tableStats;
+ private final List<Tuple> cachedData;
+ private int currentRow;
+ private boolean isClosed;
+
+ public SystemPhysicalExec(TaskAttemptContext context, ScanNode scanNode) {
+ super(context, scanNode.getInSchema(), scanNode.getOutSchema());
+ this.scanNode = scanNode;
+ this.qual = this.scanNode.getQual();
+ cachedData = TUtil.newList();
+ currentRow = 0;
+ isClosed = false;
+
+ projector = new Projector(context, inSchema, outSchema, scanNode.getTargets());
+ }
+
+ @Override
+ public Tuple next() throws IOException {
+ Tuple aTuple = null;
+ Tuple outTuple = new VTuple(outColumnNum);
+
+ if (isClosed) {
+ return null;
+ }
+
+ if (cachedData.size() == 0) {
+ rescan();
+ }
+
+ if (!scanNode.hasQual()) {
+ if (currentRow < cachedData.size()) {
+ aTuple = cachedData.get(currentRow++);
+ projector.eval(aTuple, outTuple);
+ outTuple.setOffset(aTuple.getOffset());
+ return outTuple;
+ }
+ return null;
+ } else {
+ while (currentRow < cachedData.size()) {
+ aTuple = cachedData.get(currentRow++);
+ if (qual.eval(inSchema, aTuple).isTrue()) {
+ projector.eval(aTuple, outTuple);
+ return outTuple;
+ }
+ }
+ return null;
+ }
+ }
+
+ @Override
+ public void rescan() throws IOException {
+ cachedData.clear();
+ cachedData.addAll(fetchSystemTable(scanNode.getTableDesc(), inSchema));
+
+ tableStats = new TableStats();
+ tableStats.setNumRows(cachedData.size());
+ }
+
+ @Override
+ public void close() throws IOException {
+ scanNode = null;
+ qual = null;
+ projector = null;
+ cachedData.clear();
+ currentRow = -1;
+ isClosed = true;
+ }
+
+ @Override
+ public float getProgress() {
+ return 1.0f;
+ }
+
+ @Override
+ protected void compile() throws CompilationError {
+ if (scanNode.hasQual()) {
+ qual = context.getPrecompiledEval(inSchema, qual);
+ }
+ }
+
+ @Override
+ public TableStats getInputStats() {
+ return tableStats;
+ }
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
index 2242445..2fbebc1 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
@@ -41,15 +41,12 @@ import org.apache.tajo.engine.planner.physical.StoreTableExec;
import org.apache.tajo.engine.query.QueryContext;
import org.apache.tajo.ipc.ClientProtos;
import org.apache.tajo.ipc.ClientProtos.SubmitQueryResponse;
-import org.apache.tajo.master.NonForwardQueryResultFileScanner;
-import org.apache.tajo.master.NonForwardQueryResultScanner;
-import org.apache.tajo.master.NonForwardQueryResultSystemScanner;
-import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.*;
import org.apache.tajo.master.exec.prehook.CreateTableHook;
import org.apache.tajo.master.exec.prehook.DistributedQueryHookManager;
import org.apache.tajo.master.exec.prehook.InsertIntoHook;
-import org.apache.tajo.master.querymaster.*;
-import org.apache.tajo.master.session.Session;
+import org.apache.tajo.querymaster.*;
+import org.apache.tajo.session.Session;
import org.apache.tajo.plan.LogicalPlan;
import org.apache.tajo.plan.PlanningException;
import org.apache.tajo.plan.Target;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/ha/HAService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/ha/HAService.java b/tajo-core/src/main/java/org/apache/tajo/master/ha/HAService.java
deleted file mode 100644
index 3d6669c..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/ha/HAService.java
+++ /dev/null
@@ -1,56 +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.tajo.master.ha;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * The HAService is responsible for setting active TajoMaster on startup or when the
- * current active is changing (eg due to failure), monitoring the health of TajoMaster.
- *
- */
-public interface HAService {
-
- /**
- * Add master name to shared storage.
- */
- public void register() throws IOException;
-
-
- /**
- * Delete master name to shared storage.
- *
- */
- public void delete() throws IOException;
-
- /**
- *
- * @return True if current master is an active master.
- */
- public boolean isActiveStatus();
-
- /**
- *
- * @return return all master list
- * @throws IOException
- */
- public List<TajoMasterInfo> getMasters() throws IOException;
-
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/ha/HAServiceHDFSImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/ha/HAServiceHDFSImpl.java b/tajo-core/src/main/java/org/apache/tajo/master/ha/HAServiceHDFSImpl.java
deleted file mode 100644
index 45219b3..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/ha/HAServiceHDFSImpl.java
+++ /dev/null
@@ -1,318 +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.tajo.master.ha;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.tajo.TajoConstants;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.ha.HAConstants;
-import org.apache.tajo.master.TajoMaster;
-import org.apache.tajo.master.TajoMaster.MasterContext;
-import org.apache.tajo.ha.HAServiceUtil;
-import org.apache.tajo.util.TUtil;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.List;
-
-/**
- * This implements HAService utilizing HDFS cluster. This saves master status to HDFS cluster.
- *
- */
-public class HAServiceHDFSImpl implements HAService {
- private static Log LOG = LogFactory.getLog(HAServiceHDFSImpl.class);
-
- private MasterContext context;
- private TajoConf conf;
-
- private FileSystem fs;
-
- private String masterName;
- private Path rootPath;
- private Path haPath;
- private Path activePath;
- private Path backupPath;
-
- private boolean isActiveStatus = false;
-
- //thread which runs periodically to see the last time since a heartbeat is received.
- private Thread checkerThread;
- private volatile boolean stopped = false;
-
- private int monitorInterval;
-
- private String currentActiveMaster;
-
- public HAServiceHDFSImpl(MasterContext context) throws IOException {
- this.context = context;
- this.conf = context.getConf();
- initSystemDirectory();
-
- InetSocketAddress socketAddress = context.getTajoMasterService().getBindAddress();
- this.masterName = socketAddress.getAddress().getHostAddress() + ":" + socketAddress.getPort();
-
- monitorInterval = conf.getIntVar(TajoConf.ConfVars.TAJO_MASTER_HA_MONITOR_INTERVAL);
- }
-
- private void initSystemDirectory() throws IOException {
- // Get Tajo root dir
- this.rootPath = TajoConf.getTajoRootDir(conf);
-
- // Check Tajo root dir
- this.fs = rootPath.getFileSystem(conf);
-
- // Check and create Tajo system HA dir
- haPath = TajoConf.getSystemHADir(conf);
- if (!fs.exists(haPath)) {
- fs.mkdirs(haPath, new FsPermission(TajoMaster.SYSTEM_RESOURCE_DIR_PERMISSION));
- LOG.info("System HA dir '" + haPath + "' is created");
- }
-
- activePath = new Path(haPath, TajoConstants.SYSTEM_HA_ACTIVE_DIR_NAME);
- if (!fs.exists(activePath)) {
- fs.mkdirs(activePath, new FsPermission(TajoMaster.SYSTEM_RESOURCE_DIR_PERMISSION));
- LOG.info("System HA Active dir '" + activePath + "' is created");
- }
-
- backupPath = new Path(haPath, TajoConstants.SYSTEM_HA_BACKUP_DIR_NAME);
- if (!fs.exists(backupPath)) {
- fs.mkdirs(backupPath, new FsPermission(TajoMaster.SYSTEM_RESOURCE_DIR_PERMISSION));
- LOG.info("System HA Backup dir '" + backupPath + "' is created");
- }
- }
-
- private void startPingChecker() {
- if (checkerThread == null) {
- checkerThread = new Thread(new PingChecker());
- checkerThread.setName("Ping Checker");
- checkerThread.start();
- }
- }
-
- @Override
- public void register() throws IOException {
- FileStatus[] files = fs.listStatus(activePath);
-
- // Phase 1: If there is not another active master, this try to become active master.
- if (files.length == 0) {
- createMasterFile(true);
- currentActiveMaster = masterName;
- LOG.info(String.format("This is added to active master (%s)", masterName));
- } else {
- // Phase 2: If there is active master information, we need to check its status.
- Path activePath = files[0].getPath();
- currentActiveMaster = activePath.getName().replaceAll("_", ":");
-
- // Phase 3: If current active master is dead, this master should be active master.
- if (!HAServiceUtil.isMasterAlive(currentActiveMaster, conf)) {
- fs.delete(activePath, true);
- createMasterFile(true);
- currentActiveMaster = masterName;
- LOG.info(String.format("This is added to active master (%s)", masterName));
- } else {
- // Phase 4: If current active master is alive, this master need to be backup master.
- createMasterFile(false);
- LOG.info(String.format("This is added to backup masters (%s)", masterName));
- }
- }
- }
-
- private void createMasterFile(boolean isActive) throws IOException {
- String fileName = masterName.replaceAll(":", "_");
- Path path = null;
-
- if (isActive) {
- path = new Path(activePath, fileName);
- } else {
- path = new Path(backupPath, fileName);
- }
-
- StringBuilder sb = new StringBuilder();
- InetSocketAddress address = getHostAddress(HAConstants.MASTER_CLIENT_RPC_ADDRESS);
- sb.append(address.getAddress().getHostAddress()).append(":").append(address.getPort()).append("_");
-
- address = getHostAddress(HAConstants.RESOURCE_TRACKER_RPC_ADDRESS);
- sb.append(address.getAddress().getHostAddress()).append(":").append(address.getPort()).append("_");
-
- address = getHostAddress(HAConstants.CATALOG_ADDRESS);
- sb.append(address.getAddress().getHostAddress()).append(":").append(address.getPort()).append("_");
-
- address = getHostAddress(HAConstants.MASTER_INFO_ADDRESS);
- sb.append(address.getAddress().getHostAddress()).append(":").append(address.getPort());
-
- FSDataOutputStream out = fs.create(path);
-
- try {
- out.writeUTF(sb.toString());
- out.hflush();
- out.close();
- } catch (FileAlreadyExistsException e) {
- createMasterFile(false);
- }
-
- if (isActive) {
- isActiveStatus = true;
- } else {
- isActiveStatus = false;
- }
-
- startPingChecker();
- }
-
-
- private InetSocketAddress getHostAddress(int type) {
- InetSocketAddress address = null;
-
- switch (type) {
- case HAConstants.MASTER_UMBILICAL_RPC_ADDRESS:
- address = context.getConf().getSocketAddrVar(TajoConf.ConfVars
- .TAJO_MASTER_UMBILICAL_RPC_ADDRESS);
- break;
- case HAConstants.MASTER_CLIENT_RPC_ADDRESS:
- address = context.getConf().getSocketAddrVar(TajoConf.ConfVars
- .TAJO_MASTER_CLIENT_RPC_ADDRESS);
- break;
- case HAConstants.RESOURCE_TRACKER_RPC_ADDRESS:
- address = context.getConf().getSocketAddrVar(TajoConf.ConfVars
- .RESOURCE_TRACKER_RPC_ADDRESS);
- break;
- case HAConstants.CATALOG_ADDRESS:
- address = context.getConf().getSocketAddrVar(TajoConf.ConfVars
- .CATALOG_ADDRESS);
- break;
- case HAConstants.MASTER_INFO_ADDRESS:
- address = context.getConf().getSocketAddrVar(TajoConf.ConfVars
- .TAJO_MASTER_INFO_ADDRESS);
- default:
- break;
- }
-
- return NetUtils.createSocketAddr(masterName.split(":")[0] + ":" + address.getPort());
- }
-
- @Override
- public void delete() throws IOException {
- String fileName = masterName.replaceAll(":", "_");
-
- Path activeFile = new Path(activePath, fileName);
- if (fs.exists(activeFile)) {
- fs.delete(activeFile, true);
- }
-
- Path backupFile = new Path(backupPath, fileName);
- if (fs.exists(backupFile)) {
- fs.delete(backupFile, true);
- }
- if (isActiveStatus) {
- isActiveStatus = false;
- }
- stopped = true;
- }
-
- @Override
- public boolean isActiveStatus() {
- return isActiveStatus;
- }
-
- @Override
- public List<TajoMasterInfo> getMasters() throws IOException {
- List<TajoMasterInfo> list = TUtil.newList();
- Path path = null;
-
- FileStatus[] files = fs.listStatus(activePath);
- if (files.length == 1) {
- path = files[0].getPath();
- list.add(createTajoMasterInfo(path, true));
- }
-
- files = fs.listStatus(backupPath);
- for (FileStatus status : files) {
- path = status.getPath();
- list.add(createTajoMasterInfo(path, false));
- }
-
- return list;
- }
-
- private TajoMasterInfo createTajoMasterInfo(Path path, boolean isActive) throws IOException {
- String masterAddress = path.getName().replaceAll("_", ":");
- boolean isAlive = HAServiceUtil.isMasterAlive(masterAddress, conf);
-
- FSDataInputStream stream = fs.open(path);
- String data = stream.readUTF();
-
- stream.close();
-
- String[] addresses = data.split("_");
- TajoMasterInfo info = new TajoMasterInfo();
-
- info.setTajoMasterAddress(NetUtils.createSocketAddr(masterAddress));
- info.setTajoClientAddress(NetUtils.createSocketAddr(addresses[0]));
- info.setWorkerResourceTrackerAddr(NetUtils.createSocketAddr(addresses[1]));
- info.setCatalogAddress(NetUtils.createSocketAddr(addresses[2]));
- info.setWebServerAddress(NetUtils.createSocketAddr(addresses[3]));
-
- info.setAvailable(isAlive);
- info.setActive(isActive);
-
- return info;
- }
-
- private class PingChecker implements Runnable {
- @Override
- public void run() {
- while (!stopped && !Thread.currentThread().isInterrupted()) {
- synchronized (HAServiceHDFSImpl.this) {
- try {
- if (!currentActiveMaster.equals(masterName)) {
- boolean isAlive = HAServiceUtil.isMasterAlive(currentActiveMaster, conf);
- if (LOG.isDebugEnabled()) {
- LOG.debug("currentActiveMaster:" + currentActiveMaster + ", thisMasterName:" + masterName
- + ", isAlive:" + isAlive);
- }
-
- // If active master is dead, this master should be active master instead of
- // previous active master.
- if (!isAlive) {
- FileStatus[] files = fs.listStatus(activePath);
- if (files.length == 0 || (files.length == 1
- && currentActiveMaster.equals(files[0].getPath().getName().replaceAll("_", ":")))) {
- delete();
- register();
- }
- }
- }
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- try {
- Thread.sleep(monitorInterval);
- } catch (InterruptedException e) {
- LOG.info("PingChecker interrupted. - masterName:" + masterName);
- break;
- }
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/ha/TajoMasterInfo.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/ha/TajoMasterInfo.java b/tajo-core/src/main/java/org/apache/tajo/master/ha/TajoMasterInfo.java
deleted file mode 100644
index 6ed975a..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/ha/TajoMasterInfo.java
+++ /dev/null
@@ -1,89 +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.tajo.master.ha;
-
-import java.net.InetSocketAddress;
-
-public class TajoMasterInfo {
-
- private boolean available;
- private boolean isActive;
-
- private InetSocketAddress tajoMasterAddress;
- private InetSocketAddress tajoClientAddress;
- private InetSocketAddress workerResourceTrackerAddr;
- private InetSocketAddress catalogAddress;
- private InetSocketAddress webServerAddress;
-
- public InetSocketAddress getTajoMasterAddress() {
- return tajoMasterAddress;
- }
-
- public void setTajoMasterAddress(InetSocketAddress tajoMasterAddress) {
- this.tajoMasterAddress = tajoMasterAddress;
- }
-
- public InetSocketAddress getTajoClientAddress() {
- return tajoClientAddress;
- }
-
- public void setTajoClientAddress(InetSocketAddress tajoClientAddress) {
- this.tajoClientAddress = tajoClientAddress;
- }
-
- public InetSocketAddress getWorkerResourceTrackerAddr() {
- return workerResourceTrackerAddr;
- }
-
- public void setWorkerResourceTrackerAddr(InetSocketAddress workerResourceTrackerAddr) {
- this.workerResourceTrackerAddr = workerResourceTrackerAddr;
- }
-
- public InetSocketAddress getCatalogAddress() {
- return catalogAddress;
- }
-
- public void setCatalogAddress(InetSocketAddress catalogAddress) {
- this.catalogAddress = catalogAddress;
- }
-
- public InetSocketAddress getWebServerAddress() {
- return webServerAddress;
- }
-
- public void setWebServerAddress(InetSocketAddress webServerAddress) {
- this.webServerAddress = webServerAddress;
- }
-
- public boolean isAvailable() {
- return available;
- }
-
- public void setAvailable(boolean available) {
- this.available = available;
- }
-
- public boolean isActive() {
- return isActive;
- }
-
- public void setActive(boolean active) {
- isActive = active;
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/metrics/CatalogMetricsGaugeSet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/metrics/CatalogMetricsGaugeSet.java b/tajo-core/src/main/java/org/apache/tajo/master/metrics/CatalogMetricsGaugeSet.java
deleted file mode 100644
index 7c3d283..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/metrics/CatalogMetricsGaugeSet.java
+++ /dev/null
@@ -1,56 +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.tajo.master.metrics;
-
-import com.codahale.metrics.Gauge;
-import com.codahale.metrics.Metric;
-import com.codahale.metrics.MetricSet;
-import org.apache.tajo.master.TajoMaster;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
-
-public class CatalogMetricsGaugeSet implements MetricSet {
- TajoMaster.MasterContext tajoMasterContext;
- public CatalogMetricsGaugeSet(TajoMaster.MasterContext tajoMasterContext) {
- this.tajoMasterContext = tajoMasterContext;
- }
-
- @Override
- public Map<String, Metric> getMetrics() {
- Map<String, Metric> metricsMap = new HashMap<String, Metric>();
- metricsMap.put("numTables", new Gauge<Integer>() {
- @Override
- public Integer getValue() {
- return tajoMasterContext.getCatalog().getAllTableNames(DEFAULT_DATABASE_NAME).size();
- }
- });
-
- metricsMap.put("numFunctions", new Gauge<Integer>() {
- @Override
- public Integer getValue() {
- return tajoMasterContext.getCatalog().getFunctions().size();
- }
- });
-
- return metricsMap;
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/metrics/WorkerResourceMetricsGaugeSet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/metrics/WorkerResourceMetricsGaugeSet.java b/tajo-core/src/main/java/org/apache/tajo/master/metrics/WorkerResourceMetricsGaugeSet.java
deleted file mode 100644
index 993d3b7..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/metrics/WorkerResourceMetricsGaugeSet.java
+++ /dev/null
@@ -1,74 +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.tajo.master.metrics;
-
-import com.codahale.metrics.Gauge;
-import com.codahale.metrics.Metric;
-import com.codahale.metrics.MetricSet;
-import org.apache.tajo.master.TajoMaster;
-import org.apache.tajo.master.rm.Worker;
-import org.apache.tajo.master.rm.WorkerState;
-
-import java.util.HashMap;
-import java.util.Map;
-
-public class WorkerResourceMetricsGaugeSet implements MetricSet {
- TajoMaster.MasterContext tajoMasterContext;
- public WorkerResourceMetricsGaugeSet(TajoMaster.MasterContext tajoMasterContext) {
- this.tajoMasterContext = tajoMasterContext;
- }
-
- @Override
- public Map<String, Metric> getMetrics() {
- Map<String, Metric> metricsMap = new HashMap<String, Metric>();
- metricsMap.put("totalWorkers", new Gauge<Integer>() {
- @Override
- public Integer getValue() {
- return tajoMasterContext.getResourceManager().getWorkers().size();
- }
- });
-
- metricsMap.put("liveWorkers", new Gauge<Integer>() {
- @Override
- public Integer getValue() {
- return getNumWorkers(WorkerState.RUNNING);
- }
- });
-
- metricsMap.put("deadWorkers", new Gauge<Integer>() {
- @Override
- public Integer getValue() {
- return getNumWorkers(WorkerState.LOST);
- }
- });
-
- return metricsMap;
- }
-
- protected int getNumWorkers(WorkerState status) {
- int numWorkers = 0;
- for(Worker eachWorker: tajoMasterContext.getResourceManager().getWorkers().values()) {
- if(eachWorker.getState() == status) {
- numWorkers++;
- }
- }
-
- return numWorkers;
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
deleted file mode 100644
index a626df1..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Query.java
+++ /dev/null
@@ -1,738 +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.tajo.master.querymaster;
-
-import com.google.common.collect.Maps;
-import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.state.*;
-import org.apache.hadoop.yarn.util.Clock;
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.SessionVars;
-import org.apache.tajo.TajoProtos.QueryState;
-import org.apache.tajo.catalog.proto.CatalogProtos.UpdateTableStatsProto;
-import org.apache.tajo.catalog.CatalogService;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.planner.global.ExecutionBlock;
-import org.apache.tajo.engine.planner.global.ExecutionBlockCursor;
-import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.plan.logical.*;
-import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.plan.util.PlannerUtil;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.StorageConstants;
-import org.apache.tajo.util.TUtil;
-import org.apache.tajo.util.history.QueryHistory;
-import org.apache.tajo.util.history.StageHistory;
-
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-public class Query implements EventHandler<QueryEvent> {
- private static final Log LOG = LogFactory.getLog(Query.class);
-
- // Facilities for Query
- private final TajoConf systemConf;
- private final Clock clock;
- private String queryStr;
- private Map<ExecutionBlockId, Stage> stages;
- private final EventHandler eventHandler;
- private final MasterPlan plan;
- QueryMasterTask.QueryMasterTaskContext context;
- private ExecutionBlockCursor cursor;
-
- // Query Status
- private final QueryId id;
- private long appSubmitTime;
- private long startTime;
- private long finishTime;
- private TableDesc resultDesc;
- private int completedStagesCount = 0;
- private int successedStagesCount = 0;
- private int killedStagesCount = 0;
- private int failedStagesCount = 0;
- private int erroredStagesCount = 0;
- private final List<String> diagnostics = new ArrayList<String>();
-
- // Internal Variables
- private final Lock readLock;
- private final Lock writeLock;
- private int priority = 100;
-
- // State Machine
- private final StateMachine<QueryState, QueryEventType, QueryEvent> stateMachine;
- private QueryState queryState;
-
- // Transition Handler
- private static final SingleArcTransition INTERNAL_ERROR_TRANSITION = new InternalErrorTransition();
- private static final DiagnosticsUpdateTransition DIAGNOSTIC_UPDATE_TRANSITION = new DiagnosticsUpdateTransition();
- private static final StageCompletedTransition STAGE_COMPLETED_TRANSITION = new StageCompletedTransition();
- private static final QueryCompletedTransition QUERY_COMPLETED_TRANSITION = new QueryCompletedTransition();
-
- protected static final StateMachineFactory
- <Query,QueryState,QueryEventType,QueryEvent> stateMachineFactory =
- new StateMachineFactory<Query, QueryState, QueryEventType, QueryEvent>
- (QueryState.QUERY_NEW)
-
- // Transitions from NEW state
- .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_RUNNING,
- QueryEventType.START,
- new StartTransition())
- .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_NEW,
- QueryEventType.DIAGNOSTIC_UPDATE,
- DIAGNOSTIC_UPDATE_TRANSITION)
- .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_KILLED,
- QueryEventType.KILL,
- new KillNewQueryTransition())
- .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_ERROR,
- QueryEventType.INTERNAL_ERROR,
- INTERNAL_ERROR_TRANSITION)
-
- // Transitions from RUNNING state
- .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_RUNNING,
- QueryEventType.STAGE_COMPLETED,
- STAGE_COMPLETED_TRANSITION)
- .addTransition(QueryState.QUERY_RUNNING,
- EnumSet.of(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_FAILED, QueryState.QUERY_KILLED,
- QueryState.QUERY_ERROR),
- QueryEventType.QUERY_COMPLETED,
- QUERY_COMPLETED_TRANSITION)
- .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_RUNNING,
- QueryEventType.DIAGNOSTIC_UPDATE,
- DIAGNOSTIC_UPDATE_TRANSITION)
- .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_KILL_WAIT,
- QueryEventType.KILL,
- new KillAllStagesTransition())
- .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_ERROR,
- QueryEventType.INTERNAL_ERROR,
- INTERNAL_ERROR_TRANSITION)
-
- // Transitions from QUERY_SUCCEEDED state
- .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_SUCCEEDED,
- QueryEventType.DIAGNOSTIC_UPDATE,
- DIAGNOSTIC_UPDATE_TRANSITION)
- // ignore-able transitions
- .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_SUCCEEDED,
- QueryEventType.STAGE_COMPLETED,
- STAGE_COMPLETED_TRANSITION)
- .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_SUCCEEDED,
- QueryEventType.KILL)
- .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_ERROR,
- QueryEventType.INTERNAL_ERROR,
- INTERNAL_ERROR_TRANSITION)
-
- // Transitions from KILL_WAIT state
- .addTransition(QueryState.QUERY_KILL_WAIT, QueryState.QUERY_KILL_WAIT,
- QueryEventType.STAGE_COMPLETED,
- STAGE_COMPLETED_TRANSITION)
- .addTransition(QueryState.QUERY_KILL_WAIT,
- EnumSet.of(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_FAILED, QueryState.QUERY_KILLED,
- QueryState.QUERY_ERROR),
- QueryEventType.QUERY_COMPLETED,
- QUERY_COMPLETED_TRANSITION)
- .addTransition(QueryState.QUERY_KILL_WAIT, QueryState.QUERY_KILL_WAIT,
- QueryEventType.DIAGNOSTIC_UPDATE,
- DIAGNOSTIC_UPDATE_TRANSITION)
- .addTransition(QueryState.QUERY_KILL_WAIT, QueryState.QUERY_ERROR,
- QueryEventType.INTERNAL_ERROR,
- INTERNAL_ERROR_TRANSITION)
- // Ignore-able transitions
- .addTransition(QueryState.QUERY_KILL_WAIT, EnumSet.of(QueryState.QUERY_KILLED),
- QueryEventType.KILL,
- QUERY_COMPLETED_TRANSITION)
-
- // Transitions from FAILED state
- .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_FAILED,
- QueryEventType.DIAGNOSTIC_UPDATE,
- DIAGNOSTIC_UPDATE_TRANSITION)
- .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_ERROR,
- QueryEventType.INTERNAL_ERROR,
- INTERNAL_ERROR_TRANSITION)
- // Ignore-able transitions
- .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_FAILED,
- QueryEventType.KILL)
-
- // Transitions from ERROR state
- .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
- QueryEventType.DIAGNOSTIC_UPDATE,
- DIAGNOSTIC_UPDATE_TRANSITION)
- .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
- QueryEventType.INTERNAL_ERROR,
- INTERNAL_ERROR_TRANSITION)
- // Ignore-able transitions
- .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
- EnumSet.of(QueryEventType.KILL, QueryEventType.STAGE_COMPLETED))
-
- .installTopology();
-
- public Query(final QueryMasterTask.QueryMasterTaskContext context, final QueryId id,
- final long appSubmitTime,
- final String queryStr,
- final EventHandler eventHandler,
- final MasterPlan plan) {
- this.context = context;
- this.systemConf = context.getConf();
- this.id = id;
- this.clock = context.getClock();
- this.appSubmitTime = appSubmitTime;
- this.queryStr = queryStr;
- this.stages = Maps.newConcurrentMap();
- this.eventHandler = eventHandler;
- this.plan = plan;
- this.cursor = new ExecutionBlockCursor(plan, true);
-
- StringBuilder sb = new StringBuilder("\n=======================================================");
- sb.append("\nThe order of execution: \n");
- int order = 1;
- while (cursor.hasNext()) {
- ExecutionBlock currentEB = cursor.nextBlock();
- sb.append("\n").append(order).append(": ").append(currentEB.getId());
- order++;
- }
- sb.append("\n=======================================================");
- LOG.info(sb);
- cursor.reset();
-
- ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
- this.readLock = readWriteLock.readLock();
- this.writeLock = readWriteLock.writeLock();
-
- stateMachine = stateMachineFactory.make(this);
- queryState = stateMachine.getCurrentState();
- }
-
- public float getProgress() {
- QueryState state = getState();
- if (state == QueryState.QUERY_SUCCEEDED) {
- return 1.0f;
- } else {
- int idx = 0;
- List<Stage> tempStages = new ArrayList<Stage>();
- synchronized(stages) {
- tempStages.addAll(stages.values());
- }
-
- float [] subProgresses = new float[tempStages.size()];
- for (Stage stage: tempStages) {
- if (stage.getState() != StageState.NEW) {
- subProgresses[idx] = stage.getProgress();
- } else {
- subProgresses[idx] = 0.0f;
- }
- idx++;
- }
-
- float totalProgress = 0.0f;
- float proportion = 1.0f / (float)(getExecutionBlockCursor().size() - 1); // minus one is due to
-
- for (int i = 0; i < subProgresses.length; i++) {
- totalProgress += subProgresses[i] * proportion;
- }
-
- return totalProgress;
- }
- }
-
- public long getAppSubmitTime() {
- return this.appSubmitTime;
- }
-
- public long getStartTime() {
- return startTime;
- }
-
- public void setStartTime() {
- startTime = clock.getTime();
- }
-
- public long getFinishTime() {
- return finishTime;
- }
-
- public void setFinishTime() {
- finishTime = clock.getTime();
- }
-
- public QueryHistory getQueryHistory() {
- QueryHistory queryHistory = makeQueryHistory();
- queryHistory.setStageHistories(makeStageHistories());
- return queryHistory;
- }
-
- private List<StageHistory> makeStageHistories() {
- List<StageHistory> stageHistories = new ArrayList<StageHistory>();
- for(Stage eachStage : getStages()) {
- stageHistories.add(eachStage.getStageHistory());
- }
-
- return stageHistories;
- }
-
- private QueryHistory makeQueryHistory() {
- QueryHistory queryHistory = new QueryHistory();
-
- queryHistory.setQueryId(getId().toString());
- queryHistory.setQueryMaster(context.getQueryMasterContext().getWorkerContext().getWorkerName());
- queryHistory.setHttpPort(context.getQueryMasterContext().getWorkerContext().getConnectionInfo().getHttpInfoPort());
- queryHistory.setLogicalPlan(plan.toString());
- queryHistory.setLogicalPlan(plan.getLogicalPlan().toString());
- queryHistory.setDistributedPlan(plan.toString());
-
- List<String[]> sessionVariables = new ArrayList<String[]>();
- for(Map.Entry<String,String> entry: plan.getContext().getAllKeyValus().entrySet()) {
- if (SessionVars.exists(entry.getKey()) && SessionVars.isPublic(SessionVars.get(entry.getKey()))) {
- sessionVariables.add(new String[]{entry.getKey(), entry.getValue()});
- }
- }
- queryHistory.setSessionVariables(sessionVariables);
-
- return queryHistory;
- }
-
- public List<String> getDiagnostics() {
- readLock.lock();
- try {
- return diagnostics;
- } finally {
- readLock.unlock();
- }
- }
-
- protected void addDiagnostic(String diag) {
- diagnostics.add(diag);
- }
-
- public TableDesc getResultDesc() {
- return resultDesc;
- }
-
- public void setResultDesc(TableDesc desc) {
- resultDesc = desc;
- }
-
- public MasterPlan getPlan() {
- return plan;
- }
-
- public StateMachine<QueryState, QueryEventType, QueryEvent> getStateMachine() {
- return stateMachine;
- }
-
- public void addStage(Stage stage) {
- stages.put(stage.getId(), stage);
- }
-
- public QueryId getId() {
- return this.id;
- }
-
- public Stage getStage(ExecutionBlockId id) {
- return this.stages.get(id);
- }
-
- public Collection<Stage> getStages() {
- return this.stages.values();
- }
-
- public QueryState getSynchronizedState() {
- readLock.lock();
- try {
- return stateMachine.getCurrentState();
- } finally {
- readLock.unlock();
- }
- }
-
- /* non-blocking call for client API */
- public QueryState getState() {
- return queryState;
- }
-
- public ExecutionBlockCursor getExecutionBlockCursor() {
- return cursor;
- }
-
- public static class StartTransition
- implements SingleArcTransition<Query, QueryEvent> {
-
- @Override
- public void transition(Query query, QueryEvent queryEvent) {
-
- query.setStartTime();
- Stage stage = new Stage(query.context, query.getPlan(),
- query.getExecutionBlockCursor().nextBlock());
- stage.setPriority(query.priority--);
- query.addStage(stage);
-
- stage.handle(new StageEvent(stage.getId(), StageEventType.SQ_INIT));
- LOG.debug("Schedule unit plan: \n" + stage.getBlock().getPlan());
- }
- }
-
- public static class QueryCompletedTransition implements MultipleArcTransition<Query, QueryEvent, QueryState> {
-
- @Override
- public QueryState transition(Query query, QueryEvent queryEvent) {
- QueryCompletedEvent stageEvent = (QueryCompletedEvent) queryEvent;
- QueryState finalState;
-
- if (stageEvent.getState() == StageState.SUCCEEDED) {
- finalState = finalizeQuery(query, stageEvent);
- } else if (stageEvent.getState() == StageState.FAILED) {
- finalState = QueryState.QUERY_FAILED;
- } else if (stageEvent.getState() == StageState.KILLED) {
- finalState = QueryState.QUERY_KILLED;
- } else {
- finalState = QueryState.QUERY_ERROR;
- }
- if (finalState != QueryState.QUERY_SUCCEEDED) {
- Stage lastStage = query.getStage(stageEvent.getExecutionBlockId());
- if (lastStage != null && lastStage.getTableMeta() != null) {
- StoreType storeType = lastStage.getTableMeta().getStoreType();
- if (storeType != null) {
- LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
- try {
- StorageManager.getStorageManager(query.systemConf, storeType).rollbackOutputCommit(rootNode.getChild());
- } catch (IOException e) {
- LOG.warn(query.getId() + ", failed processing cleanup storage when query failed:" + e.getMessage(), e);
- }
- }
- }
- }
- query.eventHandler.handle(new QueryMasterQueryCompletedEvent(query.getId()));
- query.setFinishTime();
-
- return finalState;
- }
-
- private QueryState finalizeQuery(Query query, QueryCompletedEvent event) {
- Stage lastStage = query.getStage(event.getExecutionBlockId());
- StoreType storeType = lastStage.getTableMeta().getStoreType();
- try {
- LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
- CatalogService catalog = lastStage.getContext().getQueryMasterContext().getWorkerContext().getCatalog();
- TableDesc tableDesc = PlannerUtil.getTableDesc(catalog, rootNode.getChild());
-
- Path finalOutputDir = StorageManager.getStorageManager(query.systemConf, storeType)
- .commitOutputData(query.context.getQueryContext(),
- lastStage.getId(), lastStage.getMasterPlan().getLogicalPlan(), lastStage.getSchema(), tableDesc);
-
- QueryHookExecutor hookExecutor = new QueryHookExecutor(query.context.getQueryMasterContext());
- hookExecutor.execute(query.context.getQueryContext(), query, event.getExecutionBlockId(), finalOutputDir);
- } catch (Exception e) {
- query.eventHandler.handle(new QueryDiagnosticsUpdateEvent(query.id, ExceptionUtils.getStackTrace(e)));
- return QueryState.QUERY_ERROR;
- }
-
- return QueryState.QUERY_SUCCEEDED;
- }
-
- private static interface QueryHook {
- boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir);
- void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext, Query query,
- ExecutionBlockId finalExecBlockId, Path finalOutputDir) throws Exception;
- }
-
- private class QueryHookExecutor {
- private List<QueryHook> hookList = TUtil.newList();
- private QueryMaster.QueryMasterContext context;
-
- public QueryHookExecutor(QueryMaster.QueryMasterContext context) {
- this.context = context;
- hookList.add(new MaterializedResultHook());
- hookList.add(new CreateTableHook());
- hookList.add(new InsertTableHook());
- }
-
- public void execute(QueryContext queryContext, Query query,
- ExecutionBlockId finalExecBlockId,
- Path finalOutputDir) throws Exception {
- for (QueryHook hook : hookList) {
- if (hook.isEligible(queryContext, query, finalExecBlockId, finalOutputDir)) {
- hook.execute(context, queryContext, query, finalExecBlockId, finalOutputDir);
- }
- }
- }
- }
-
- private class MaterializedResultHook implements QueryHook {
-
- @Override
- public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
- Path finalOutputDir) {
- Stage lastStage = query.getStage(finalExecBlockId);
- NodeType type = lastStage.getBlock().getPlan().getType();
- return type != NodeType.CREATE_TABLE && type != NodeType.INSERT;
- }
-
- @Override
- public void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext,
- Query query, ExecutionBlockId finalExecBlockId,
- Path finalOutputDir) throws Exception {
- Stage lastStage = query.getStage(finalExecBlockId);
- TableMeta meta = lastStage.getTableMeta();
-
- String nullChar = queryContext.get(SessionVars.NULL_CHAR);
- meta.putOption(StorageConstants.TEXT_NULL, nullChar);
-
- TableStats stats = lastStage.getResultStats();
-
- TableDesc resultTableDesc =
- new TableDesc(
- query.getId().toString(),
- lastStage.getSchema(),
- meta,
- finalOutputDir.toUri());
- resultTableDesc.setExternal(true);
-
- stats.setNumBytes(getTableVolume(query.systemConf, finalOutputDir));
- resultTableDesc.setStats(stats);
- query.setResultDesc(resultTableDesc);
- }
- }
-
- private class CreateTableHook implements QueryHook {
-
- @Override
- public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
- Path finalOutputDir) {
- Stage lastStage = query.getStage(finalExecBlockId);
- return lastStage.getBlock().getPlan().getType() == NodeType.CREATE_TABLE;
- }
-
- @Override
- public void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext,
- Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir) throws Exception {
- CatalogService catalog = context.getWorkerContext().getCatalog();
- Stage lastStage = query.getStage(finalExecBlockId);
- TableStats stats = lastStage.getResultStats();
-
- CreateTableNode createTableNode = (CreateTableNode) lastStage.getBlock().getPlan();
- TableMeta meta = new TableMeta(createTableNode.getStorageType(), createTableNode.getOptions());
-
- TableDesc tableDescTobeCreated =
- new TableDesc(
- createTableNode.getTableName(),
- createTableNode.getTableSchema(),
- meta,
- finalOutputDir.toUri());
- tableDescTobeCreated.setExternal(createTableNode.isExternal());
-
- if (createTableNode.hasPartition()) {
- tableDescTobeCreated.setPartitionMethod(createTableNode.getPartitionMethod());
- }
-
- stats.setNumBytes(getTableVolume(query.systemConf, finalOutputDir));
- tableDescTobeCreated.setStats(stats);
- query.setResultDesc(tableDescTobeCreated);
-
- catalog.createTable(tableDescTobeCreated);
- }
- }
-
- private class InsertTableHook implements QueryHook {
-
- @Override
- public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
- Path finalOutputDir) {
- Stage lastStage = query.getStage(finalExecBlockId);
- return lastStage.getBlock().getPlan().getType() == NodeType.INSERT;
- }
-
- @Override
- public void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext,
- Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir)
- throws Exception {
-
- CatalogService catalog = context.getWorkerContext().getCatalog();
- Stage lastStage = query.getStage(finalExecBlockId);
- TableMeta meta = lastStage.getTableMeta();
- TableStats stats = lastStage.getResultStats();
-
- InsertNode insertNode = (InsertNode) lastStage.getBlock().getPlan();
-
- TableDesc finalTable;
- if (insertNode.hasTargetTable()) {
- String tableName = insertNode.getTableName();
- finalTable = catalog.getTableDesc(tableName);
- } else {
- String tableName = query.getId().toString();
- finalTable = new TableDesc(tableName, lastStage.getSchema(), meta, finalOutputDir.toUri());
- }
-
- long volume = getTableVolume(query.systemConf, finalOutputDir);
- stats.setNumBytes(volume);
- finalTable.setStats(stats);
-
- if (insertNode.hasTargetTable()) {
- UpdateTableStatsProto.Builder builder = UpdateTableStatsProto.newBuilder();
- builder.setTableName(finalTable.getName());
- builder.setStats(stats.getProto());
-
- catalog.updateTableStats(builder.build());
- }
-
- query.setResultDesc(finalTable);
- }
- }
- }
-
- public static long getTableVolume(TajoConf systemConf, Path tablePath) throws IOException {
- FileSystem fs = tablePath.getFileSystem(systemConf);
- ContentSummary directorySummary = fs.getContentSummary(tablePath);
- return directorySummary.getLength();
- }
-
- public static class StageCompletedTransition implements SingleArcTransition<Query, QueryEvent> {
-
- private boolean hasNext(Query query) {
- ExecutionBlockCursor cursor = query.getExecutionBlockCursor();
- ExecutionBlock nextBlock = cursor.peek();
- return !query.getPlan().isTerminal(nextBlock);
- }
-
- private void executeNextBlock(Query query) {
- ExecutionBlockCursor cursor = query.getExecutionBlockCursor();
- ExecutionBlock nextBlock = cursor.nextBlock();
- Stage nextStage = new Stage(query.context, query.getPlan(), nextBlock);
- nextStage.setPriority(query.priority--);
- query.addStage(nextStage);
- nextStage.handle(new StageEvent(nextStage.getId(), StageEventType.SQ_INIT));
-
- LOG.info("Scheduling Stage:" + nextStage.getId());
- if(LOG.isDebugEnabled()) {
- LOG.debug("Scheduling Stage's Priority: " + nextStage.getPriority());
- LOG.debug("Scheduling Stage's Plan: \n" + nextStage.getBlock().getPlan());
- }
- }
-
- @Override
- public void transition(Query query, QueryEvent event) {
- try {
- query.completedStagesCount++;
- StageCompletedEvent castEvent = (StageCompletedEvent) event;
-
- if (castEvent.getState() == StageState.SUCCEEDED) {
- query.successedStagesCount++;
- } else if (castEvent.getState() == StageState.KILLED) {
- query.killedStagesCount++;
- } else if (castEvent.getState() == StageState.FAILED) {
- query.failedStagesCount++;
- } else if (castEvent.getState() == StageState.ERROR) {
- query.erroredStagesCount++;
- } else {
- LOG.error(String.format("Invalid Stage (%s) State %s at %s",
- castEvent.getExecutionBlockId().toString(), castEvent.getState().name(), query.getSynchronizedState().name()));
- query.eventHandler.handle(new QueryEvent(event.getQueryId(), QueryEventType.INTERNAL_ERROR));
- }
-
- // if a stage is succeeded and a query is running
- if (castEvent.getState() == StageState.SUCCEEDED && // latest stage succeeded
- query.getSynchronizedState() == QueryState.QUERY_RUNNING && // current state is not in KILL_WAIT, FAILED, or ERROR.
- hasNext(query)) { // there remains at least one stage.
- query.getStage(castEvent.getExecutionBlockId()).waitingIntermediateReport();
- executeNextBlock(query);
- } else { // if a query is completed due to finished, kill, failure, or error
- query.eventHandler.handle(new QueryCompletedEvent(castEvent.getExecutionBlockId(), castEvent.getState()));
- }
- } catch (Throwable t) {
- LOG.error(t.getMessage(), t);
- query.eventHandler.handle(new QueryEvent(event.getQueryId(), QueryEventType.INTERNAL_ERROR));
- }
- }
- }
-
- private static class DiagnosticsUpdateTransition implements SingleArcTransition<Query, QueryEvent> {
- @Override
- public void transition(Query query, QueryEvent event) {
- query.addDiagnostic(((QueryDiagnosticsUpdateEvent) event).getDiagnosticUpdate());
- }
- }
-
- private static class KillNewQueryTransition implements SingleArcTransition<Query, QueryEvent> {
- @Override
- public void transition(Query query, QueryEvent event) {
- query.setFinishTime();
- query.eventHandler.handle(new QueryMasterQueryCompletedEvent(query.getId()));
- }
- }
-
- private static class KillAllStagesTransition implements SingleArcTransition<Query, QueryEvent> {
- @Override
- public void transition(Query query, QueryEvent event) {
- synchronized (query.stages) {
- for (Stage stage : query.stages.values()) {
- query.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_KILL));
- }
- }
- }
- }
-
- private static class InternalErrorTransition implements SingleArcTransition<Query, QueryEvent> {
-
- @Override
- public void transition(Query query, QueryEvent event) {
- query.setFinishTime();
- query.eventHandler.handle(new QueryMasterQueryCompletedEvent(query.getId()));
- }
- }
-
- @Override
- public void handle(QueryEvent event) {
- LOG.info("Processing " + event.getQueryId() + " of type " + event.getType());
- try {
- writeLock.lock();
- QueryState oldState = getSynchronizedState();
- try {
- getStateMachine().doTransition(event.getType(), event);
- queryState = getSynchronizedState();
- } catch (InvalidStateTransitonException e) {
- LOG.error("Can't handle this event at current state"
- + ", type:" + event
- + ", oldState:" + oldState.name()
- + ", nextState:" + getSynchronizedState().name()
- , e);
- eventHandler.handle(new QueryEvent(this.id, QueryEventType.INTERNAL_ERROR));
- }
-
- //notify the eventhandler of state change
- if (oldState != getSynchronizedState()) {
- LOG.info(id + " Query Transitioned from " + oldState + " to " + getSynchronizedState());
- }
- }
-
- finally {
- writeLock.unlock();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
deleted file mode 100644
index 0a87990..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
+++ /dev/null
@@ -1,300 +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.tajo.master.querymaster;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.CompositeService;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.TajoProtos;
-import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.ipc.QueryMasterProtocol;
-import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService;
-import org.apache.tajo.ipc.TajoWorkerProtocol;
-import org.apache.tajo.ipc.TajoWorkerProtocol.QueryExecutionRequestProto;
-import org.apache.tajo.master.TajoMaster;
-import org.apache.tajo.master.rm.WorkerResourceManager;
-import org.apache.tajo.master.session.Session;
-import org.apache.tajo.plan.logical.LogicalRootNode;
-import org.apache.tajo.rpc.NettyClientBase;
-import org.apache.tajo.rpc.NullCallback;
-import org.apache.tajo.rpc.RpcConnectionPool;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
-import org.apache.tajo.util.NetUtils;
-
-import java.net.InetSocketAddress;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static org.apache.tajo.ipc.TajoMasterProtocol.WorkerAllocatedResource;
-
-public class QueryInProgress extends CompositeService {
- private static final Log LOG = LogFactory.getLog(QueryInProgress.class.getName());
-
- private QueryId queryId;
-
- private Session session;
-
- private AsyncDispatcher dispatcher;
-
- private LogicalRootNode plan;
-
- private AtomicBoolean querySubmitted = new AtomicBoolean(false);
-
- private AtomicBoolean stopped = new AtomicBoolean(false);
-
- private QueryInfo queryInfo;
-
- private final TajoMaster.MasterContext masterContext;
-
- private NettyClientBase queryMasterRpc;
-
- private QueryMasterProtocolService queryMasterRpcClient;
-
- public QueryInProgress(
- TajoMaster.MasterContext masterContext,
- Session session,
- QueryContext queryContext,
- QueryId queryId, String sql, String jsonExpr, LogicalRootNode plan) {
- super(QueryInProgress.class.getName());
- this.masterContext = masterContext;
- this.session = session;
- this.queryId = queryId;
- this.plan = plan;
-
- queryInfo = new QueryInfo(queryId, queryContext, sql, jsonExpr);
- queryInfo.setStartTime(System.currentTimeMillis());
- }
-
- @Override
- public void init(Configuration conf) {
- dispatcher = new AsyncDispatcher();
- this.addService(dispatcher);
-
- dispatcher.register(QueryJobEvent.Type.class, new QueryInProgressEventHandler());
- super.init(conf);
- }
-
- public synchronized void kill() {
- if(queryMasterRpcClient != null){
- queryMasterRpcClient.killQuery(null, queryId.getProto(), NullCallback.get());
- }
- }
-
- @Override
- public void stop() {
- if(stopped.getAndSet(true)) {
- return;
- }
-
- LOG.info("=========================================================");
- LOG.info("Stop query:" + queryId);
-
- masterContext.getResourceManager().stopQueryMaster(queryId);
-
- long startTime = System.currentTimeMillis();
- while(true) {
- try {
- if(masterContext.getResourceManager().isQueryMasterStopped(queryId)) {
- LOG.info(queryId + " QueryMaster stopped");
- break;
- }
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- break;
- }
-
- try {
- synchronized (this){
- wait(100);
- }
- } catch (InterruptedException e) {
- break;
- }
- if(System.currentTimeMillis() - startTime > 60 * 1000) {
- LOG.warn("Failed to stop QueryMaster:" + queryId);
- break;
- }
- }
-
- if(queryMasterRpc != null) {
- RpcConnectionPool.getPool(masterContext.getConf()).closeConnection(queryMasterRpc);
- }
-
- masterContext.getHistoryWriter().appendHistory(queryInfo);
- super.stop();
- }
-
- @Override
- public void start() {
- super.start();
- }
-
- public EventHandler getEventHandler() {
- return dispatcher.getEventHandler();
- }
-
-
-
- public boolean startQueryMaster() {
- try {
- LOG.info("Initializing QueryInProgress for QueryID=" + queryId);
- WorkerResourceManager resourceManager = masterContext.getResourceManager();
- WorkerAllocatedResource resource = resourceManager.allocateQueryMaster(this);
-
- // if no resource to allocate a query master
- if(resource == null) {
- LOG.info("No Available Resources for QueryMaster");
- return false;
- }
-
- queryInfo.setQueryMaster(resource.getConnectionInfo().getHost());
- queryInfo.setQueryMasterPort(resource.getConnectionInfo().getQueryMasterPort());
- queryInfo.setQueryMasterclientPort(resource.getConnectionInfo().getClientPort());
- queryInfo.setQueryMasterInfoPort(resource.getConnectionInfo().getHttpInfoPort());
-
- getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_MASTER_START, queryInfo));
-
- return true;
- } catch (Exception e) {
- catchException(e);
- return false;
- }
- }
-
- class QueryInProgressEventHandler implements EventHandler<QueryJobEvent> {
- @Override
- public void handle(QueryJobEvent queryJobEvent) {
- if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_HEARTBEAT) {
- heartbeat(queryJobEvent.getQueryInfo());
- } else if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_MASTER_START) {
- QueryInProgress queryInProgress = masterContext.getQueryJobManager().getQueryInProgress(queryId);
- queryInProgress.getEventHandler().handle(
- new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_START, queryInProgress.getQueryInfo()));
- } else if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_START) {
- submmitQueryToMaster();
- } else if (queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_KILL) {
- kill();
- }
- }
- }
-
- private void connectQueryMaster() throws Exception {
- InetSocketAddress addr = NetUtils.createSocketAddr(queryInfo.getQueryMasterHost(), queryInfo.getQueryMasterPort());
- LOG.info("Connect to QueryMaster:" + addr);
- queryMasterRpc =
- RpcConnectionPool.getPool(masterContext.getConf()).getConnection(addr, QueryMasterProtocol.class, true);
- queryMasterRpcClient = queryMasterRpc.getStub();
- }
-
- private synchronized void submmitQueryToMaster() {
- if(querySubmitted.get()) {
- return;
- }
-
- try {
- if(queryMasterRpcClient == null) {
- connectQueryMaster();
- }
- if(queryMasterRpcClient == null) {
- LOG.info("No QueryMaster conneciton info.");
- //TODO wait
- return;
- }
- LOG.info("Call executeQuery to :" +
- queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort() + "," + queryId);
-
- QueryExecutionRequestProto.Builder builder = TajoWorkerProtocol.QueryExecutionRequestProto.newBuilder();
- builder.setQueryId(queryId.getProto())
- .setQueryContext(queryInfo.getQueryContext().getProto())
- .setSession(session.getProto())
- .setExprInJson(PrimitiveProtos.StringProto.newBuilder().setValue(queryInfo.getJsonExpr()))
- .setLogicalPlanJson(PrimitiveProtos.StringProto.newBuilder().setValue(plan.toJson()).build());
-
- queryMasterRpcClient.executeQuery(null, builder.build(), NullCallback.get());
- querySubmitted.set(true);
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- }
- }
-
- public void catchException(Exception e) {
- LOG.error(e.getMessage(), e);
- queryInfo.setQueryState(TajoProtos.QueryState.QUERY_FAILED);
- queryInfo.setLastMessage(StringUtils.stringifyException(e));
- }
-
- public QueryId getQueryId() {
- return queryId;
- }
-
- public QueryInfo getQueryInfo() {
- return this.queryInfo;
- }
-
- public boolean isStarted() {
- return !stopped.get() && this.querySubmitted.get();
- }
-
- private void heartbeat(QueryInfo queryInfo) {
- LOG.info("Received QueryMaster heartbeat:" + queryInfo);
-
- // to avoid partial update by different heartbeats
- synchronized (this.queryInfo) {
-
- // terminal state will let client to retrieve a query result
- // So, we must set the query result before changing query state
- if (isFinishState(queryInfo.getQueryState())) {
- if (queryInfo.hasResultdesc()) {
- this.queryInfo.setResultDesc(queryInfo.getResultDesc());
- }
- }
-
- this.queryInfo.setQueryState(queryInfo.getQueryState());
- this.queryInfo.setProgress(queryInfo.getProgress());
- this.queryInfo.setFinishTime(queryInfo.getFinishTime());
-
- // Update diagnosis message
- if (queryInfo.getLastMessage() != null && !queryInfo.getLastMessage().isEmpty()) {
- this.queryInfo.setLastMessage(queryInfo.getLastMessage());
- LOG.info(queryId + queryInfo.getLastMessage());
- }
-
- // if any error occurs, print outs the error message
- if (this.queryInfo.getQueryState() == TajoProtos.QueryState.QUERY_FAILED) {
- LOG.warn(queryId + " failed, " + queryInfo.getLastMessage());
- }
-
-
- if (isFinishState(this.queryInfo.getQueryState())) {
- masterContext.getQueryJobManager().getEventHandler().handle(
- new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_STOP, this.queryInfo));
- }
- }
- }
-
- private boolean isFinishState(TajoProtos.QueryState state) {
- return state == TajoProtos.QueryState.QUERY_FAILED ||
- state == TajoProtos.QueryState.QUERY_KILLED ||
- state == TajoProtos.QueryState.QUERY_SUCCEEDED;
- }
-}
[09/16] tajo git commit: TAJO-1288: Refactoring
org.apache.tajo.master package.
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Stage.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Stage.java
deleted file mode 100644
index 0515e72..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Stage.java
+++ /dev/null
@@ -1,1342 +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.tajo.master.querymaster;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.yarn.api.records.Priority;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.event.Event;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.state.*;
-import org.apache.hadoop.yarn.util.Records;
-import org.apache.tajo.*;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.catalog.statistics.ColumnStats;
-import org.apache.tajo.catalog.statistics.StatisticsUtil;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.json.CoreGsonHelper;
-import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
-import org.apache.tajo.engine.planner.enforce.Enforcer;
-import org.apache.tajo.engine.planner.global.DataChannel;
-import org.apache.tajo.engine.planner.global.ExecutionBlock;
-import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.ipc.TajoMasterProtocol;
-import org.apache.tajo.ipc.TajoWorkerProtocol;
-import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.MultipleAggregationStage;
-import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty;
-import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto;
-import org.apache.tajo.master.*;
-import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext;
-import org.apache.tajo.master.querymaster.Task.IntermediateEntry;
-import org.apache.tajo.master.container.TajoContainer;
-import org.apache.tajo.master.container.TajoContainerId;
-import org.apache.tajo.storage.FileStorageManager;
-import org.apache.tajo.plan.util.PlannerUtil;
-import org.apache.tajo.plan.logical.*;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.unit.StorageUnit;
-import org.apache.tajo.util.KeyValueSet;
-import org.apache.tajo.util.history.TaskHistory;
-import org.apache.tajo.util.history.StageHistory;
-import org.apache.tajo.worker.FetchImpl;
-
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import static org.apache.tajo.conf.TajoConf.ConfVars;
-import static org.apache.tajo.plan.serder.PlanProto.ShuffleType;
-
-
-/**
- * Stage plays a role in controlling an ExecutionBlock and is a finite state machine.
- */
-public class Stage implements EventHandler<StageEvent> {
-
- private static final Log LOG = LogFactory.getLog(Stage.class);
-
- private MasterPlan masterPlan;
- private ExecutionBlock block;
- private int priority;
- private Schema schema;
- private TableMeta meta;
- private TableStats resultStatistics;
- private TableStats inputStatistics;
- private EventHandler<Event> eventHandler;
- private AbstractTaskScheduler taskScheduler;
- private QueryMasterTask.QueryMasterTaskContext context;
- private final List<String> diagnostics = new ArrayList<String>();
- private StageState stageState;
-
- private long startTime;
- private long finishTime;
-
- volatile Map<TaskId, Task> tasks = new ConcurrentHashMap<TaskId, Task>();
- volatile Map<TajoContainerId, TajoContainer> containers = new ConcurrentHashMap<TajoContainerId,
- TajoContainer>();
-
- private static final DiagnosticsUpdateTransition DIAGNOSTIC_UPDATE_TRANSITION = new DiagnosticsUpdateTransition();
- private static final InternalErrorTransition INTERNAL_ERROR_TRANSITION = new InternalErrorTransition();
- private static final ContainerLaunchTransition CONTAINER_LAUNCH_TRANSITION = new ContainerLaunchTransition();
- private static final TaskCompletedTransition TASK_COMPLETED_TRANSITION = new TaskCompletedTransition();
- private static final AllocatedContainersCancelTransition CONTAINERS_CANCEL_TRANSITION =
- new AllocatedContainersCancelTransition();
- private static final StageCompleteTransition STAGE_COMPLETED_TRANSITION = new StageCompleteTransition();
- private StateMachine<StageState, StageEventType, StageEvent> stateMachine;
-
- protected static final StateMachineFactory<Stage, StageState,
- StageEventType, StageEvent> stateMachineFactory =
- new StateMachineFactory <Stage, StageState,
- StageEventType, StageEvent> (StageState.NEW)
-
- // Transitions from NEW state
- .addTransition(StageState.NEW,
- EnumSet.of(StageState.INITED, StageState.ERROR, StageState.SUCCEEDED),
- StageEventType.SQ_INIT,
- new InitAndRequestContainer())
- .addTransition(StageState.NEW, StageState.NEW,
- StageEventType.SQ_DIAGNOSTIC_UPDATE,
- DIAGNOSTIC_UPDATE_TRANSITION)
- .addTransition(StageState.NEW, StageState.KILLED,
- StageEventType.SQ_KILL)
- .addTransition(StageState.NEW, StageState.ERROR,
- StageEventType.SQ_INTERNAL_ERROR,
- INTERNAL_ERROR_TRANSITION)
-
- // Transitions from INITED state
- .addTransition(StageState.INITED, StageState.RUNNING,
- StageEventType.SQ_CONTAINER_ALLOCATED,
- CONTAINER_LAUNCH_TRANSITION)
- .addTransition(StageState.INITED, StageState.INITED,
- StageEventType.SQ_DIAGNOSTIC_UPDATE,
- DIAGNOSTIC_UPDATE_TRANSITION)
- .addTransition(StageState.INITED, StageState.KILL_WAIT,
- StageEventType.SQ_KILL, new KillTasksTransition())
- .addTransition(StageState.INITED, StageState.ERROR,
- StageEventType.SQ_INTERNAL_ERROR,
- INTERNAL_ERROR_TRANSITION)
-
- // Transitions from RUNNING state
- .addTransition(StageState.RUNNING, StageState.RUNNING,
- StageEventType.SQ_CONTAINER_ALLOCATED,
- CONTAINER_LAUNCH_TRANSITION)
- .addTransition(StageState.RUNNING, StageState.RUNNING,
- StageEventType.SQ_TASK_COMPLETED,
- TASK_COMPLETED_TRANSITION)
- .addTransition(StageState.RUNNING,
- EnumSet.of(StageState.SUCCEEDED, StageState.FAILED),
- StageEventType.SQ_STAGE_COMPLETED,
- STAGE_COMPLETED_TRANSITION)
- .addTransition(StageState.RUNNING, StageState.RUNNING,
- StageEventType.SQ_FAILED,
- TASK_COMPLETED_TRANSITION)
- .addTransition(StageState.RUNNING, StageState.RUNNING,
- StageEventType.SQ_DIAGNOSTIC_UPDATE,
- DIAGNOSTIC_UPDATE_TRANSITION)
- .addTransition(StageState.RUNNING, StageState.KILL_WAIT,
- StageEventType.SQ_KILL,
- new KillTasksTransition())
- .addTransition(StageState.RUNNING, StageState.ERROR,
- StageEventType.SQ_INTERNAL_ERROR,
- INTERNAL_ERROR_TRANSITION)
- // Ignore-able Transition
- .addTransition(StageState.RUNNING, StageState.RUNNING,
- StageEventType.SQ_START)
-
- // Transitions from KILL_WAIT state
- .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
- StageEventType.SQ_CONTAINER_ALLOCATED,
- CONTAINERS_CANCEL_TRANSITION)
- .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
- EnumSet.of(StageEventType.SQ_KILL), new KillTasksTransition())
- .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
- StageEventType.SQ_TASK_COMPLETED,
- TASK_COMPLETED_TRANSITION)
- .addTransition(StageState.KILL_WAIT,
- EnumSet.of(StageState.SUCCEEDED, StageState.FAILED, StageState.KILLED),
- StageEventType.SQ_STAGE_COMPLETED,
- STAGE_COMPLETED_TRANSITION)
- .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
- StageEventType.SQ_DIAGNOSTIC_UPDATE,
- DIAGNOSTIC_UPDATE_TRANSITION)
- .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
- StageEventType.SQ_FAILED,
- TASK_COMPLETED_TRANSITION)
- .addTransition(StageState.KILL_WAIT, StageState.ERROR,
- StageEventType.SQ_INTERNAL_ERROR,
- INTERNAL_ERROR_TRANSITION)
-
- // Transitions from SUCCEEDED state
- .addTransition(StageState.SUCCEEDED, StageState.SUCCEEDED,
- StageEventType.SQ_CONTAINER_ALLOCATED,
- CONTAINERS_CANCEL_TRANSITION)
- .addTransition(StageState.SUCCEEDED, StageState.SUCCEEDED,
- StageEventType.SQ_DIAGNOSTIC_UPDATE,
- DIAGNOSTIC_UPDATE_TRANSITION)
- .addTransition(StageState.SUCCEEDED, StageState.ERROR,
- StageEventType.SQ_INTERNAL_ERROR,
- INTERNAL_ERROR_TRANSITION)
- // Ignore-able events
- .addTransition(StageState.SUCCEEDED, StageState.SUCCEEDED,
- EnumSet.of(
- StageEventType.SQ_START,
- StageEventType.SQ_KILL,
- StageEventType.SQ_CONTAINER_ALLOCATED))
-
- // Transitions from KILLED state
- .addTransition(StageState.KILLED, StageState.KILLED,
- StageEventType.SQ_CONTAINER_ALLOCATED,
- CONTAINERS_CANCEL_TRANSITION)
- .addTransition(StageState.KILLED, StageState.KILLED,
- StageEventType.SQ_DIAGNOSTIC_UPDATE,
- DIAGNOSTIC_UPDATE_TRANSITION)
- .addTransition(StageState.KILLED, StageState.ERROR,
- StageEventType.SQ_INTERNAL_ERROR,
- INTERNAL_ERROR_TRANSITION)
- // Ignore-able transitions
- .addTransition(StageState.KILLED, StageState.KILLED,
- EnumSet.of(
- StageEventType.SQ_START,
- StageEventType.SQ_KILL,
- StageEventType.SQ_CONTAINER_ALLOCATED,
- StageEventType.SQ_FAILED))
-
- // Transitions from FAILED state
- .addTransition(StageState.FAILED, StageState.FAILED,
- StageEventType.SQ_CONTAINER_ALLOCATED,
- CONTAINERS_CANCEL_TRANSITION)
- .addTransition(StageState.FAILED, StageState.FAILED,
- StageEventType.SQ_DIAGNOSTIC_UPDATE,
- DIAGNOSTIC_UPDATE_TRANSITION)
- .addTransition(StageState.FAILED, StageState.ERROR,
- StageEventType.SQ_INTERNAL_ERROR,
- INTERNAL_ERROR_TRANSITION)
- // Ignore-able transitions
- .addTransition(StageState.FAILED, StageState.FAILED,
- EnumSet.of(
- StageEventType.SQ_START,
- StageEventType.SQ_KILL,
- StageEventType.SQ_CONTAINER_ALLOCATED,
- StageEventType.SQ_FAILED))
-
- // Transitions from ERROR state
- .addTransition(StageState.ERROR, StageState.ERROR,
- StageEventType.SQ_CONTAINER_ALLOCATED,
- CONTAINERS_CANCEL_TRANSITION)
- .addTransition(StageState.ERROR, StageState.ERROR,
- StageEventType.SQ_DIAGNOSTIC_UPDATE,
- DIAGNOSTIC_UPDATE_TRANSITION)
- // Ignore-able transitions
- .addTransition(StageState.ERROR, StageState.ERROR,
- EnumSet.of(
- StageEventType.SQ_START,
- StageEventType.SQ_KILL,
- StageEventType.SQ_FAILED,
- StageEventType.SQ_INTERNAL_ERROR,
- StageEventType.SQ_STAGE_COMPLETED))
-
- .installTopology();
-
- private final Lock readLock;
- private final Lock writeLock;
-
- private int totalScheduledObjectsCount;
- private int succeededObjectCount = 0;
- private int completedTaskCount = 0;
- private int succeededTaskCount = 0;
- private int killedObjectCount = 0;
- private int failedObjectCount = 0;
- private TaskSchedulerContext schedulerContext;
- private List<IntermediateEntry> hashShuffleIntermediateEntries = new ArrayList<IntermediateEntry>();
- private AtomicInteger completeReportReceived = new AtomicInteger(0);
- private StageHistory finalStageHistory;
-
- public Stage(QueryMasterTask.QueryMasterTaskContext context, MasterPlan masterPlan, ExecutionBlock block) {
- this.context = context;
- this.masterPlan = masterPlan;
- this.block = block;
- this.eventHandler = context.getEventHandler();
-
- ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
- this.readLock = readWriteLock.readLock();
- this.writeLock = readWriteLock.writeLock();
- stateMachine = stateMachineFactory.make(this);
- stageState = stateMachine.getCurrentState();
- }
-
- public static boolean isRunningState(StageState state) {
- return state == StageState.INITED || state == StageState.NEW || state == StageState.RUNNING;
- }
-
- public QueryMasterTask.QueryMasterTaskContext getContext() {
- return context;
- }
-
- public MasterPlan getMasterPlan() {
- return masterPlan;
- }
-
- public DataChannel getDataChannel() {
- return masterPlan.getOutgoingChannels(getId()).iterator().next();
- }
-
- public EventHandler<Event> getEventHandler() {
- return eventHandler;
- }
-
- public AbstractTaskScheduler getTaskScheduler() {
- return taskScheduler;
- }
-
- public void setStartTime() {
- startTime = context.getClock().getTime();
- }
-
- @SuppressWarnings("UnusedDeclaration")
- public long getStartTime() {
- return this.startTime;
- }
-
- public void setFinishTime() {
- finishTime = context.getClock().getTime();
- }
-
- @SuppressWarnings("UnusedDeclaration")
- public long getFinishTime() {
- return this.finishTime;
- }
-
- public float getTaskProgress() {
- readLock.lock();
- try {
- if (getState() == StageState.NEW) {
- return 0;
- } else {
- return (float)(succeededObjectCount) / (float)totalScheduledObjectsCount;
- }
- } finally {
- readLock.unlock();
- }
- }
-
- public float getProgress() {
- List<Task> tempTasks = null;
- readLock.lock();
- try {
- if (getState() == StageState.NEW) {
- return 0.0f;
- } else {
- tempTasks = new ArrayList<Task>(tasks.values());
- }
- } finally {
- readLock.unlock();
- }
-
- float totalProgress = 0.0f;
- for (Task eachTask : tempTasks) {
- if (eachTask.getLastAttempt() != null) {
- totalProgress += eachTask.getLastAttempt().getProgress();
- }
- }
-
- if (totalProgress > 0.0f) {
- return (float) Math.floor((totalProgress / (float) Math.max(tempTasks.size(), 1)) * 1000.0f) / 1000.0f;
- } else {
- return 0.0f;
- }
- }
-
- public int getSucceededObjectCount() {
- return succeededObjectCount;
- }
-
- public int getTotalScheduledObjectsCount() {
- return totalScheduledObjectsCount;
- }
-
- public ExecutionBlock getBlock() {
- return block;
- }
-
- public void addTask(Task task) {
- tasks.put(task.getId(), task);
- }
-
- public StageHistory getStageHistory() {
- if (finalStageHistory != null) {
- if (finalStageHistory.getFinishTime() == 0) {
- finalStageHistory = makeStageHistory();
- finalStageHistory.setTasks(makeTaskHistories());
- }
- return finalStageHistory;
- } else {
- return makeStageHistory();
- }
- }
-
- private List<TaskHistory> makeTaskHistories() {
- List<TaskHistory> taskHistories = new ArrayList<TaskHistory>();
-
- for(Task eachTask : getTasks()) {
- taskHistories.add(eachTask.getTaskHistory());
- }
-
- return taskHistories;
- }
-
- private StageHistory makeStageHistory() {
- StageHistory stageHistory = new StageHistory();
-
- stageHistory.setExecutionBlockId(getId().toString());
- stageHistory.setPlan(PlannerUtil.buildExplainString(block.getPlan()));
- stageHistory.setState(getState().toString());
- stageHistory.setStartTime(startTime);
- stageHistory.setFinishTime(finishTime);
- stageHistory.setSucceededObjectCount(succeededObjectCount);
- stageHistory.setKilledObjectCount(killedObjectCount);
- stageHistory.setFailedObjectCount(failedObjectCount);
- stageHistory.setTotalScheduledObjectsCount(totalScheduledObjectsCount);
- stageHistory.setHostLocalAssigned(getTaskScheduler().getHostLocalAssigned());
- stageHistory.setRackLocalAssigned(getTaskScheduler().getRackLocalAssigned());
-
- long totalInputBytes = 0;
- long totalReadBytes = 0;
- long totalReadRows = 0;
- long totalWriteBytes = 0;
- long totalWriteRows = 0;
- int numShuffles = 0;
- for(Task eachTask : getTasks()) {
- numShuffles = eachTask.getShuffleOutpuNum();
- if (eachTask.getLastAttempt() != null) {
- TableStats inputStats = eachTask.getLastAttempt().getInputStats();
- if (inputStats != null) {
- totalInputBytes += inputStats.getNumBytes();
- totalReadBytes += inputStats.getReadBytes();
- totalReadRows += inputStats.getNumRows();
- }
- TableStats outputStats = eachTask.getLastAttempt().getResultStats();
- if (outputStats != null) {
- totalWriteBytes += outputStats.getNumBytes();
- totalWriteRows += outputStats.getNumRows();
- }
- }
- }
-
- stageHistory.setTotalInputBytes(totalInputBytes);
- stageHistory.setTotalReadBytes(totalReadBytes);
- stageHistory.setTotalReadRows(totalReadRows);
- stageHistory.setTotalWriteBytes(totalWriteBytes);
- stageHistory.setTotalWriteRows(totalWriteRows);
- stageHistory.setNumShuffles(numShuffles);
- stageHistory.setProgress(getProgress());
- return stageHistory;
- }
-
- /**
- * It finalizes this stage. It is only invoked when the stage is succeeded.
- */
- public void complete() {
- cleanup();
- finalizeStats();
- setFinishTime();
- eventHandler.handle(new StageCompletedEvent(getId(), StageState.SUCCEEDED));
- }
-
- /**
- * It finalizes this stage. Unlike {@link Stage#complete()},
- * it is invoked when a stage is abnormally finished.
- *
- * @param finalState The final stage state
- */
- public void abort(StageState finalState) {
- // TODO -
- // - committer.abortStage(...)
- // - record Stage Finish Time
- // - CleanUp Tasks
- // - Record History
- cleanup();
- setFinishTime();
- eventHandler.handle(new StageCompletedEvent(getId(), finalState));
- }
-
- public StateMachine<StageState, StageEventType, StageEvent> getStateMachine() {
- return this.stateMachine;
- }
-
- public void setPriority(int priority) {
- this.priority = priority;
- }
-
-
- public int getPriority() {
- return this.priority;
- }
-
- public ExecutionBlockId getId() {
- return block.getId();
- }
-
- public Task[] getTasks() {
- return tasks.values().toArray(new Task[tasks.size()]);
- }
-
- public Task getTask(TaskId qid) {
- return tasks.get(qid);
- }
-
- public Schema getSchema() {
- return schema;
- }
-
- public TableMeta getTableMeta() {
- return meta;
- }
-
- public TableStats getResultStats() {
- return resultStatistics;
- }
-
- public TableStats getInputStats() {
- return inputStatistics;
- }
-
- public List<String> getDiagnostics() {
- readLock.lock();
- try {
- return diagnostics;
- } finally {
- readLock.unlock();
- }
- }
-
- protected void addDiagnostic(String diag) {
- diagnostics.add(diag);
- }
-
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append(this.getId());
- return sb.toString();
- }
-
- @Override
- public boolean equals(Object o) {
- if (o instanceof Stage) {
- Stage other = (Stage)o;
- return getId().equals(other.getId());
- }
- return false;
- }
-
- @Override
- public int hashCode() {
- return getId().hashCode();
- }
-
- public int compareTo(Stage other) {
- return getId().compareTo(other.getId());
- }
-
- public StageState getSynchronizedState() {
- readLock.lock();
- try {
- return stateMachine.getCurrentState();
- } finally {
- readLock.unlock();
- }
- }
-
- /* non-blocking call for client API */
- public StageState getState() {
- return stageState;
- }
-
- public static TableStats[] computeStatFromUnionBlock(Stage stage) {
- TableStats[] stat = new TableStats[]{new TableStats(), new TableStats()};
- long[] avgRows = new long[]{0, 0};
- long[] numBytes = new long[]{0, 0};
- long[] readBytes = new long[]{0, 0};
- long[] numRows = new long[]{0, 0};
- int[] numBlocks = new int[]{0, 0};
- int[] numOutputs = new int[]{0, 0};
-
- List<ColumnStats> columnStatses = Lists.newArrayList();
-
- MasterPlan masterPlan = stage.getMasterPlan();
- Iterator<ExecutionBlock> it = masterPlan.getChilds(stage.getBlock()).iterator();
- while (it.hasNext()) {
- ExecutionBlock block = it.next();
- Stage childStage = stage.context.getStage(block.getId());
- TableStats[] childStatArray = new TableStats[]{
- childStage.getInputStats(), childStage.getResultStats()
- };
- for (int i = 0; i < 2; i++) {
- if (childStatArray[i] == null) {
- continue;
- }
- avgRows[i] += childStatArray[i].getAvgRows();
- numBlocks[i] += childStatArray[i].getNumBlocks();
- numBytes[i] += childStatArray[i].getNumBytes();
- readBytes[i] += childStatArray[i].getReadBytes();
- numOutputs[i] += childStatArray[i].getNumShuffleOutputs();
- numRows[i] += childStatArray[i].getNumRows();
- }
- columnStatses.addAll(childStatArray[1].getColumnStats());
- }
-
- for (int i = 0; i < 2; i++) {
- stat[i].setNumBlocks(numBlocks[i]);
- stat[i].setNumBytes(numBytes[i]);
- stat[i].setReadBytes(readBytes[i]);
- stat[i].setNumShuffleOutputs(numOutputs[i]);
- stat[i].setNumRows(numRows[i]);
- stat[i].setAvgRows(avgRows[i]);
- }
- stat[1].setColumnStats(columnStatses);
-
- return stat;
- }
-
- private TableStats[] computeStatFromTasks() {
- List<TableStats> inputStatsList = Lists.newArrayList();
- List<TableStats> resultStatsList = Lists.newArrayList();
- for (Task unit : getTasks()) {
- resultStatsList.add(unit.getStats());
- if (unit.getLastAttempt().getInputStats() != null) {
- inputStatsList.add(unit.getLastAttempt().getInputStats());
- }
- }
- TableStats inputStats = StatisticsUtil.aggregateTableStat(inputStatsList);
- TableStats resultStats = StatisticsUtil.aggregateTableStat(resultStatsList);
- return new TableStats[]{inputStats, resultStats};
- }
-
- private void stopScheduler() {
- // If there are launched TaskRunners, send the 'shouldDie' message to all r
- // via received task requests.
- if (taskScheduler != null) {
- taskScheduler.stop();
- }
- }
-
- private void releaseContainers() {
- // If there are still live TaskRunners, try to kill the containers.
- eventHandler.handle(new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP, getId(), containers.values()));
- }
-
- /**
- * It computes all stats and sets the intermediate result.
- */
- private void finalizeStats() {
- TableStats[] statsArray;
- if (block.hasUnion()) {
- statsArray = computeStatFromUnionBlock(this);
- } else {
- statsArray = computeStatFromTasks();
- }
-
- DataChannel channel = masterPlan.getOutgoingChannels(getId()).get(0);
-
- // if store plan (i.e., CREATE or INSERT OVERWRITE)
- StoreType storeType = PlannerUtil.getStoreType(masterPlan.getLogicalPlan());
- if (storeType == null) {
- // get default or store type
- storeType = StoreType.CSV;
- }
-
- schema = channel.getSchema();
- meta = CatalogUtil.newTableMeta(storeType, new KeyValueSet());
- inputStatistics = statsArray[0];
- resultStatistics = statsArray[1];
- }
-
- @Override
- public void handle(StageEvent event) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Processing " + event.getStageId() + " of type " + event.getType() + ", preState="
- + getSynchronizedState());
- }
-
- try {
- writeLock.lock();
- StageState oldState = getSynchronizedState();
- try {
- getStateMachine().doTransition(event.getType(), event);
- stageState = getSynchronizedState();
- } catch (InvalidStateTransitonException e) {
- LOG.error("Can't handle this event at current state"
- + ", eventType:" + event.getType().name()
- + ", oldState:" + oldState.name()
- + ", nextState:" + getSynchronizedState().name()
- , e);
- eventHandler.handle(new StageEvent(getId(),
- StageEventType.SQ_INTERNAL_ERROR));
- }
-
- // notify the eventhandler of state change
- if (LOG.isDebugEnabled()) {
- if (oldState != getSynchronizedState()) {
- LOG.debug(getId() + " Stage Transitioned from " + oldState + " to "
- + getSynchronizedState());
- }
- }
- } finally {
- writeLock.unlock();
- }
- }
-
- public void handleTaskRequestEvent(TaskRequestEvent event) {
- taskScheduler.handleTaskRequestEvent(event);
- }
-
- private static class InitAndRequestContainer implements MultipleArcTransition<Stage,
- StageEvent, StageState> {
-
- @Override
- public StageState transition(final Stage stage, StageEvent stageEvent) {
- stage.setStartTime();
- ExecutionBlock execBlock = stage.getBlock();
- StageState state;
-
- try {
- // Union operator does not require actual query processing. It is performed logically.
- if (execBlock.hasUnion()) {
- stage.finalizeStats();
- state = StageState.SUCCEEDED;
- } else {
- ExecutionBlock parent = stage.getMasterPlan().getParent(stage.getBlock());
- DataChannel channel = stage.getMasterPlan().getChannel(stage.getId(), parent.getId());
- setShuffleIfNecessary(stage, channel);
- initTaskScheduler(stage);
- // execute pre-processing asyncronously
- stage.getContext().getQueryMasterContext().getEventExecutor()
- .submit(new Runnable() {
- @Override
- public void run() {
- try {
- schedule(stage);
- stage.totalScheduledObjectsCount = stage.getTaskScheduler().remainingScheduledObjectNum();
- LOG.info(stage.totalScheduledObjectsCount + " objects are scheduled");
-
- if (stage.getTaskScheduler().remainingScheduledObjectNum() == 0) { // if there is no tasks
- stage.complete();
- } else {
- if(stage.getSynchronizedState() == StageState.INITED) {
- stage.taskScheduler.start();
- allocateContainers(stage);
- } else {
- stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_KILL));
- }
- }
- } catch (Throwable e) {
- LOG.error("Stage (" + stage.getId() + ") ERROR: ", e);
- stage.setFinishTime();
- stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(), e.getMessage()));
- stage.eventHandler.handle(new StageCompletedEvent(stage.getId(), StageState.ERROR));
- }
- }
- }
- );
- state = StageState.INITED;
- }
- } catch (Throwable e) {
- LOG.error("Stage (" + stage.getId() + ") ERROR: ", e);
- stage.setFinishTime();
- stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(), e.getMessage()));
- stage.eventHandler.handle(new StageCompletedEvent(stage.getId(), StageState.ERROR));
- return StageState.ERROR;
- }
-
- return state;
- }
-
- private void initTaskScheduler(Stage stage) throws IOException {
- TajoConf conf = stage.context.getConf();
- stage.schedulerContext = new TaskSchedulerContext(stage.context,
- stage.getMasterPlan().isLeaf(stage.getId()), stage.getId());
- stage.taskScheduler = TaskSchedulerFactory.get(conf, stage.schedulerContext, stage);
- stage.taskScheduler.init(conf);
- LOG.info(stage.taskScheduler.getName() + " is chosen for the task scheduling for " + stage.getId());
- }
-
- /**
- * If a parent block requires a repartition operation, the method sets proper repartition
- * methods and the number of partitions to a given Stage.
- */
- private static void setShuffleIfNecessary(Stage stage, DataChannel channel) {
- if (channel.getShuffleType() != ShuffleType.NONE_SHUFFLE) {
- int numTasks = calculateShuffleOutputNum(stage, channel);
- Repartitioner.setShuffleOutputNumForTwoPhase(stage, numTasks, channel);
- }
- }
-
- /**
- * Getting the total memory of cluster
- *
- * @param stage
- * @return mega bytes
- */
- private static int getClusterTotalMemory(Stage stage) {
- List<TajoMasterProtocol.WorkerResourceProto> workers =
- stage.context.getQueryMasterContext().getQueryMaster().getAllWorker();
-
- int totalMem = 0;
- for (TajoMasterProtocol.WorkerResourceProto worker : workers) {
- totalMem += worker.getMemoryMB();
- }
- return totalMem;
- }
- /**
- * Getting the desire number of partitions according to the volume of input data.
- * This method is only used to determine the partition key number of hash join or aggregation.
- *
- * @param stage
- * @return
- */
- public static int calculateShuffleOutputNum(Stage stage, DataChannel channel) {
- TajoConf conf = stage.context.getConf();
- MasterPlan masterPlan = stage.getMasterPlan();
- ExecutionBlock parent = masterPlan.getParent(stage.getBlock());
-
- LogicalNode grpNode = null;
- if (parent != null) {
- grpNode = PlannerUtil.findMostBottomNode(parent.getPlan(), NodeType.GROUP_BY);
- if (grpNode == null) {
- grpNode = PlannerUtil.findMostBottomNode(parent.getPlan(), NodeType.DISTINCT_GROUP_BY);
- }
- }
-
- // We assume this execution block the first stage of join if two or more tables are included in this block,
- if (parent != null && parent.getScanNodes().length >= 2) {
- List<ExecutionBlock> childs = masterPlan.getChilds(parent);
-
- // for outer
- ExecutionBlock outer = childs.get(0);
- long outerVolume = getInputVolume(stage.masterPlan, stage.context, outer);
-
- // for inner
- ExecutionBlock inner = childs.get(1);
- long innerVolume = getInputVolume(stage.masterPlan, stage.context, inner);
- LOG.info(stage.getId() + ", Outer volume: " + Math.ceil((double) outerVolume / 1048576) + "MB, "
- + "Inner volume: " + Math.ceil((double) innerVolume / 1048576) + "MB");
-
- long bigger = Math.max(outerVolume, innerVolume);
-
- int mb = (int) Math.ceil((double) bigger / 1048576);
- LOG.info(stage.getId() + ", Bigger Table's volume is approximately " + mb + " MB");
-
- int taskNum = (int) Math.ceil((double) mb / masterPlan.getContext().getInt(SessionVars.JOIN_PER_SHUFFLE_SIZE));
-
- if (masterPlan.getContext().containsKey(SessionVars.TEST_MIN_TASK_NUM)) {
- taskNum = masterPlan.getContext().getInt(SessionVars.TEST_MIN_TASK_NUM);
- LOG.warn("!!!!! TESTCASE MODE !!!!!");
- }
-
- // The shuffle output numbers of join may be inconsistent by execution block order.
- // Thus, we need to compare the number with DataChannel output numbers.
- // If the number is right, the number and DataChannel output numbers will be consistent.
- int outerShuffleOutputNum = 0, innerShuffleOutputNum = 0;
- for (DataChannel eachChannel : masterPlan.getOutgoingChannels(outer.getId())) {
- outerShuffleOutputNum = Math.max(outerShuffleOutputNum, eachChannel.getShuffleOutputNum());
- }
- for (DataChannel eachChannel : masterPlan.getOutgoingChannels(inner.getId())) {
- innerShuffleOutputNum = Math.max(innerShuffleOutputNum, eachChannel.getShuffleOutputNum());
- }
- if (outerShuffleOutputNum != innerShuffleOutputNum
- && taskNum != outerShuffleOutputNum
- && taskNum != innerShuffleOutputNum) {
- LOG.info(stage.getId() + ", Change determined number of join partitions cause difference of outputNum" +
- ", originTaskNum=" + taskNum + ", changedTaskNum=" + Math.max(outerShuffleOutputNum, innerShuffleOutputNum) +
- ", outerShuffleOutptNum=" + outerShuffleOutputNum +
- ", innerShuffleOutputNum=" + innerShuffleOutputNum);
- taskNum = Math.max(outerShuffleOutputNum, innerShuffleOutputNum);
- }
-
- LOG.info(stage.getId() + ", The determined number of join partitions is " + taskNum);
-
- return taskNum;
- // Is this stage the first step of group-by?
- } else if (grpNode != null) {
- boolean hasGroupColumns = true;
- if (grpNode.getType() == NodeType.GROUP_BY) {
- hasGroupColumns = ((GroupbyNode)grpNode).getGroupingColumns().length > 0;
- } else if (grpNode.getType() == NodeType.DISTINCT_GROUP_BY) {
- // Find current distinct stage node.
- DistinctGroupbyNode distinctNode = PlannerUtil.findMostBottomNode(stage.getBlock().getPlan(), NodeType.DISTINCT_GROUP_BY);
- if (distinctNode == null) {
- LOG.warn(stage.getId() + ", Can't find current DistinctGroupbyNode");
- distinctNode = (DistinctGroupbyNode)grpNode;
- }
- hasGroupColumns = distinctNode.getGroupingColumns().length > 0;
-
- Enforcer enforcer = stage.getBlock().getEnforcer();
- if (enforcer == null) {
- LOG.warn(stage.getId() + ", DistinctGroupbyNode's enforcer is null.");
- }
- EnforceProperty property = PhysicalPlannerImpl.getAlgorithmEnforceProperty(enforcer, distinctNode);
- if (property != null) {
- if (property.getDistinct().getIsMultipleAggregation()) {
- MultipleAggregationStage multiAggStage = property.getDistinct().getMultipleAggregationStage();
- if (multiAggStage != MultipleAggregationStage.THRID_STAGE) {
- hasGroupColumns = true;
- }
- }
- }
- }
- if (!hasGroupColumns) {
- LOG.info(stage.getId() + ", No Grouping Column - determinedTaskNum is set to 1");
- return 1;
- } else {
- long volume = getInputVolume(stage.masterPlan, stage.context, stage.block);
-
- int volumeByMB = (int) Math.ceil((double) volume / StorageUnit.MB);
- LOG.info(stage.getId() + ", Table's volume is approximately " + volumeByMB + " MB");
- // determine the number of task
- int taskNum = (int) Math.ceil((double) volumeByMB /
- masterPlan.getContext().getInt(SessionVars.GROUPBY_PER_SHUFFLE_SIZE));
- LOG.info(stage.getId() + ", The determined number of aggregation partitions is " + taskNum);
- return taskNum;
- }
- } else {
- LOG.info("============>>>>> Unexpected Case! <<<<<================");
- long volume = getInputVolume(stage.masterPlan, stage.context, stage.block);
-
- int mb = (int) Math.ceil((double)volume / 1048576);
- LOG.info(stage.getId() + ", Table's volume is approximately " + mb + " MB");
- // determine the number of task per 128MB
- int taskNum = (int) Math.ceil((double)mb / 128);
- LOG.info(stage.getId() + ", The determined number of partitions is " + taskNum);
- return taskNum;
- }
- }
-
- private static void schedule(Stage stage) throws IOException {
- MasterPlan masterPlan = stage.getMasterPlan();
- ExecutionBlock execBlock = stage.getBlock();
- if (stage.getMasterPlan().isLeaf(execBlock.getId()) && execBlock.getScanNodes().length == 1) { // Case 1: Just Scan
- scheduleFragmentsForLeafQuery(stage);
- } else if (execBlock.getScanNodes().length > 1) { // Case 2: Join
- Repartitioner.scheduleFragmentsForJoinQuery(stage.schedulerContext, stage);
- } else { // Case 3: Others (Sort or Aggregation)
- int numTasks = getNonLeafTaskNum(stage);
- Repartitioner.scheduleFragmentsForNonLeafTasks(stage.schedulerContext, masterPlan, stage, numTasks);
- }
- }
-
- /**
- * Getting the desire number of tasks according to the volume of input data
- *
- * @param stage
- * @return
- */
- public static int getNonLeafTaskNum(Stage stage) {
- // Getting intermediate data size
- long volume = getInputVolume(stage.getMasterPlan(), stage.context, stage.getBlock());
-
- int mb = (int) Math.ceil((double)volume / 1048576);
- LOG.info(stage.getId() + ", Table's volume is approximately " + mb + " MB");
- // determine the number of task per 64MB
- int maxTaskNum = Math.max(1, (int) Math.ceil((double)mb / 64));
- LOG.info(stage.getId() + ", The determined number of non-leaf tasks is " + maxTaskNum);
- return maxTaskNum;
- }
-
- public static long getInputVolume(MasterPlan masterPlan, QueryMasterTask.QueryMasterTaskContext context,
- ExecutionBlock execBlock) {
- Map<String, TableDesc> tableMap = context.getTableDescMap();
- if (masterPlan.isLeaf(execBlock)) {
- ScanNode[] outerScans = execBlock.getScanNodes();
- long maxVolume = 0;
- for (ScanNode eachScanNode: outerScans) {
- TableStats stat = tableMap.get(eachScanNode.getCanonicalName()).getStats();
- if (stat.getNumBytes() > maxVolume) {
- maxVolume = stat.getNumBytes();
- }
- }
- return maxVolume;
- } else {
- long aggregatedVolume = 0;
- for (ExecutionBlock childBlock : masterPlan.getChilds(execBlock)) {
- Stage stage = context.getStage(childBlock.getId());
- if (stage == null || stage.getSynchronizedState() != StageState.SUCCEEDED) {
- aggregatedVolume += getInputVolume(masterPlan, context, childBlock);
- } else {
- aggregatedVolume += stage.getResultStats().getNumBytes();
- }
- }
-
- return aggregatedVolume;
- }
- }
-
- public static void allocateContainers(Stage stage) {
- ExecutionBlock execBlock = stage.getBlock();
-
- //TODO consider disk slot
- int requiredMemoryMBPerTask = 512;
-
- int numRequest = stage.getContext().getResourceAllocator().calculateNumRequestContainers(
- stage.getContext().getQueryMasterContext().getWorkerContext(),
- stage.schedulerContext.getEstimatedTaskNum(),
- requiredMemoryMBPerTask
- );
-
- final Resource resource = Records.newRecord(Resource.class);
-
- resource.setMemory(requiredMemoryMBPerTask);
-
- LOG.info("Request Container for " + stage.getId() + " containers=" + numRequest);
-
- Priority priority = Records.newRecord(Priority.class);
- priority.setPriority(stage.getPriority());
- ContainerAllocationEvent event =
- new ContainerAllocationEvent(ContainerAllocatorEventType.CONTAINER_REQ,
- stage.getId(), priority, resource, numRequest,
- stage.masterPlan.isLeaf(execBlock), 0.0f);
- stage.eventHandler.handle(event);
- }
-
- private static void scheduleFragmentsForLeafQuery(Stage stage) throws IOException {
- ExecutionBlock execBlock = stage.getBlock();
- ScanNode[] scans = execBlock.getScanNodes();
- Preconditions.checkArgument(scans.length == 1, "Must be Scan Query");
- ScanNode scan = scans[0];
- TableDesc table = stage.context.getTableDescMap().get(scan.getCanonicalName());
-
- Collection<Fragment> fragments;
- TableMeta meta = table.getMeta();
-
- // Depending on scanner node's type, it creates fragments. If scan is for
- // a partitioned table, It will creates lots fragments for all partitions.
- // Otherwise, it creates at least one fragments for a table, which may
- // span a number of blocks or possibly consists of a number of files.
- if (scan.getType() == NodeType.PARTITIONS_SCAN) {
- // After calling this method, partition paths are removed from the physical plan.
- FileStorageManager storageManager =
- (FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf());
- fragments = Repartitioner.getFragmentsFromPartitionedTable(storageManager, scan, table);
- } else {
- StorageManager storageManager =
- StorageManager.getStorageManager(stage.getContext().getConf(), meta.getStoreType());
- fragments = storageManager.getSplits(scan.getCanonicalName(), table, scan);
- }
-
- Stage.scheduleFragments(stage, fragments);
- if (stage.getTaskScheduler() instanceof DefaultTaskScheduler) {
- //Leaf task of DefaultTaskScheduler should be fragment size
- // EstimatedTaskNum determined number of initial container
- stage.schedulerContext.setEstimatedTaskNum(fragments.size());
- } else {
- TajoConf conf = stage.context.getConf();
- stage.schedulerContext.setTaskSize(conf.getIntVar(ConfVars.TASK_DEFAULT_SIZE) * 1024 * 1024);
- int estimatedTaskNum = (int) Math.ceil((double) table.getStats().getNumBytes() /
- (double) stage.schedulerContext.getTaskSize());
- stage.schedulerContext.setEstimatedTaskNum(estimatedTaskNum);
- }
- }
- }
-
- public static void scheduleFragment(Stage stage, Fragment fragment) {
- stage.taskScheduler.handle(new FragmentScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
- stage.getId(), fragment));
- }
-
-
- public static void scheduleFragments(Stage stage, Collection<Fragment> fragments) {
- for (Fragment eachFragment : fragments) {
- scheduleFragment(stage, eachFragment);
- }
- }
-
- public static void scheduleFragments(Stage stage, Collection<Fragment> leftFragments,
- Collection<Fragment> broadcastFragments) {
- for (Fragment eachLeafFragment : leftFragments) {
- scheduleFragment(stage, eachLeafFragment, broadcastFragments);
- }
- }
-
- public static void scheduleFragment(Stage stage,
- Fragment leftFragment, Collection<Fragment> rightFragments) {
- stage.taskScheduler.handle(new FragmentScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
- stage.getId(), leftFragment, rightFragments));
- }
-
- public static void scheduleFetches(Stage stage, Map<String, List<FetchImpl>> fetches) {
- stage.taskScheduler.handle(new FetchScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
- stage.getId(), fetches));
- }
-
- public static Task newEmptyTask(TaskSchedulerContext schedulerContext,
- TaskAttemptScheduleContext taskContext,
- Stage stage, int taskId) {
- ExecutionBlock execBlock = stage.getBlock();
- Task unit = new Task(schedulerContext.getMasterContext().getConf(),
- taskContext,
- QueryIdFactory.newTaskId(schedulerContext.getBlockId(), taskId),
- schedulerContext.isLeafQuery(), stage.eventHandler);
- unit.setLogicalPlan(execBlock.getPlan());
- stage.addTask(unit);
- return unit;
- }
-
- private static class ContainerLaunchTransition
- implements SingleArcTransition<Stage, StageEvent> {
-
- @Override
- public void transition(Stage stage, StageEvent event) {
- try {
- StageContainerAllocationEvent allocationEvent =
- (StageContainerAllocationEvent) event;
- for (TajoContainer container : allocationEvent.getAllocatedContainer()) {
- TajoContainerId cId = container.getId();
- if (stage.containers.containsKey(cId)) {
- stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(),
- "Duplicated containers are allocated: " + cId.toString()));
- stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_INTERNAL_ERROR));
- }
- stage.containers.put(cId, container);
- }
- LOG.info("Stage (" + stage.getId() + ") has " + stage.containers.size() + " containers!");
- stage.eventHandler.handle(
- new LaunchTaskRunnersEvent(stage.getId(), allocationEvent.getAllocatedContainer(),
- stage.getContext().getQueryContext(),
- CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class))
- );
-
- stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_START));
- } catch (Throwable t) {
- stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(),
- ExceptionUtils.getStackTrace(t)));
- stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_INTERNAL_ERROR));
- }
- }
- }
-
- /**
- * It is used in KILL_WAIT state against Contained Allocated event.
- * It just returns allocated containers to resource manager.
- */
- private static class AllocatedContainersCancelTransition implements SingleArcTransition<Stage, StageEvent> {
- @Override
- public void transition(Stage stage, StageEvent event) {
- try {
- StageContainerAllocationEvent allocationEvent =
- (StageContainerAllocationEvent) event;
- stage.eventHandler.handle(
- new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP,
- stage.getId(), allocationEvent.getAllocatedContainer()));
- LOG.info(String.format("[%s] %d allocated containers are canceled",
- stage.getId().toString(),
- allocationEvent.getAllocatedContainer().size()));
- } catch (Throwable t) {
- stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(),
- ExceptionUtils.getStackTrace(t)));
- stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_INTERNAL_ERROR));
- }
- }
- }
-
- private static class TaskCompletedTransition implements SingleArcTransition<Stage, StageEvent> {
-
- @Override
- public void transition(Stage stage,
- StageEvent event) {
- StageTaskEvent taskEvent = (StageTaskEvent) event;
- Task task = stage.getTask(taskEvent.getTaskId());
-
- if (task == null) { // task failed
- LOG.error(String.format("Task %s is absent", taskEvent.getTaskId()));
- stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_FAILED));
- } else {
- stage.completedTaskCount++;
-
- if (taskEvent.getState() == TaskState.SUCCEEDED) {
- stage.succeededObjectCount++;
- } else if (task.getState() == TaskState.KILLED) {
- stage.killedObjectCount++;
- } else if (task.getState() == TaskState.FAILED) {
- stage.failedObjectCount++;
- // if at least one task is failed, try to kill all tasks.
- stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_KILL));
- }
-
- LOG.info(String.format("[%s] Task Completion Event (Total: %d, Success: %d, Killed: %d, Failed: %d)",
- stage.getId(),
- stage.getTotalScheduledObjectsCount(),
- stage.succeededObjectCount,
- stage.killedObjectCount,
- stage.failedObjectCount));
-
- if (stage.totalScheduledObjectsCount ==
- stage.succeededObjectCount + stage.killedObjectCount + stage.failedObjectCount) {
- stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_STAGE_COMPLETED));
- }
- }
- }
- }
-
- private static class KillTasksTransition implements SingleArcTransition<Stage, StageEvent> {
-
- @Override
- public void transition(Stage stage, StageEvent stageEvent) {
- if(stage.getTaskScheduler() != null){
- stage.getTaskScheduler().stop();
- }
-
- for (Task task : stage.getTasks()) {
- stage.eventHandler.handle(new TaskEvent(task.getId(), TaskEventType.T_KILL));
- }
- }
- }
-
- private void cleanup() {
- stopScheduler();
- releaseContainers();
-
- if (!getContext().getQueryContext().getBool(SessionVars.DEBUG_ENABLED)) {
- List<ExecutionBlock> childs = getMasterPlan().getChilds(getId());
- List<TajoIdProtos.ExecutionBlockIdProto> ebIds = Lists.newArrayList();
-
- for (ExecutionBlock executionBlock : childs) {
- ebIds.add(executionBlock.getId().getProto());
- }
-
- getContext().getQueryMasterContext().getQueryMaster().cleanupExecutionBlock(ebIds);
- }
-
- this.finalStageHistory = makeStageHistory();
- this.finalStageHistory.setTasks(makeTaskHistories());
- }
-
- public List<IntermediateEntry> getHashShuffleIntermediateEntries() {
- return hashShuffleIntermediateEntries;
- }
-
- protected void waitingIntermediateReport() {
- LOG.info(getId() + ", waiting IntermediateReport: expectedTaskNum=" + completeReportReceived.get());
- synchronized(completeReportReceived) {
- long startTime = System.currentTimeMillis();
- while (true) {
- if (completeReportReceived.get() >= tasks.size()) {
- LOG.info(getId() + ", completed waiting IntermediateReport");
- return;
- } else {
- try {
- completeReportReceived.wait(10 * 1000);
- } catch (InterruptedException e) {
- }
- long elapsedTime = System.currentTimeMillis() - startTime;
- if (elapsedTime >= 120 * 1000) {
- LOG.error(getId() + ", Timeout while receiving intermediate reports: " + elapsedTime + " ms");
- abort(StageState.FAILED);
- return;
- }
- }
- }
- }
- }
-
- public void receiveExecutionBlockReport(TajoWorkerProtocol.ExecutionBlockReport report) {
- LOG.info(getId() + ", receiveExecutionBlockReport:" + report.getSucceededTasks());
- if (!report.getReportSuccess()) {
- LOG.error(getId() + ", ExecutionBlock final report fail cause:" + report.getReportErrorMessage());
- abort(StageState.FAILED);
- return;
- }
- if (report.getIntermediateEntriesCount() > 0) {
- synchronized (hashShuffleIntermediateEntries) {
- for (IntermediateEntryProto eachInterm: report.getIntermediateEntriesList()) {
- hashShuffleIntermediateEntries.add(new IntermediateEntry(eachInterm));
- }
- }
- }
- synchronized(completeReportReceived) {
- completeReportReceived.addAndGet(report.getSucceededTasks());
- completeReportReceived.notifyAll();
- }
- }
-
- private static class StageCompleteTransition implements MultipleArcTransition<Stage, StageEvent, StageState> {
-
- @Override
- public StageState transition(Stage stage, StageEvent stageEvent) {
- // TODO - Commit Stage
- // TODO - records succeeded, failed, killed completed task
- // TODO - records metrics
- try {
- LOG.info(String.format("Stage completed - %s (total=%d, success=%d, killed=%d)",
- stage.getId().toString(),
- stage.getTotalScheduledObjectsCount(),
- stage.getSucceededObjectCount(),
- stage.killedObjectCount));
-
- if (stage.killedObjectCount > 0 || stage.failedObjectCount > 0) {
- if (stage.failedObjectCount > 0) {
- stage.abort(StageState.FAILED);
- return StageState.FAILED;
- } else if (stage.killedObjectCount > 0) {
- stage.abort(StageState.KILLED);
- return StageState.KILLED;
- } else {
- LOG.error("Invalid State " + stage.getSynchronizedState() + " State");
- stage.abort(StageState.ERROR);
- return StageState.ERROR;
- }
- } else {
- stage.complete();
- return StageState.SUCCEEDED;
- }
- } catch (Throwable t) {
- LOG.error(t.getMessage(), t);
- stage.abort(StageState.ERROR);
- return StageState.ERROR;
- }
- }
- }
-
- private static class DiagnosticsUpdateTransition implements SingleArcTransition<Stage, StageEvent> {
- @Override
- public void transition(Stage stage, StageEvent event) {
- stage.addDiagnostic(((StageDiagnosticsUpdateEvent) event).getDiagnosticUpdate());
- }
- }
-
- private static class InternalErrorTransition implements SingleArcTransition<Stage, StageEvent> {
- @Override
- public void transition(Stage stage, StageEvent stageEvent) {
- stage.abort(StageState.ERROR);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/StageState.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/StageState.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/StageState.java
deleted file mode 100644
index 82a06fe..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/StageState.java
+++ /dev/null
@@ -1,30 +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.tajo.master.querymaster;
-
-public enum StageState {
- NEW,
- INITED,
- RUNNING,
- SUCCEEDED,
- FAILED,
- KILL_WAIT,
- KILLED,
- ERROR
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Task.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Task.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Task.java
deleted file mode 100644
index 5475791..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Task.java
+++ /dev/null
@@ -1,907 +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.tajo.master.querymaster;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Objects;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.state.*;
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.QueryIdFactory;
-import org.apache.tajo.TaskAttemptId;
-import org.apache.tajo.TaskId;
-import org.apache.tajo.TajoProtos.TaskAttemptState;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.ipc.TajoWorkerProtocol.FailureIntermediateProto;
-import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto;
-import org.apache.tajo.master.FragmentPair;
-import org.apache.tajo.master.TaskState;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext;
-import org.apache.tajo.plan.logical.*;
-import org.apache.tajo.storage.DataLocation;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.storage.fragment.FragmentConvertor;
-import org.apache.tajo.util.Pair;
-import org.apache.tajo.util.TajoIdUtils;
-import org.apache.tajo.util.history.TaskHistory;
-import org.apache.tajo.worker.FetchImpl;
-
-import java.net.URI;
-import java.util.*;
-import java.util.Map.Entry;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
-import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleFileOutput;
-
-public class Task implements EventHandler<TaskEvent> {
- /** Class Logger */
- private static final Log LOG = LogFactory.getLog(Task.class);
-
- private final Configuration systemConf;
- private TaskId taskId;
- private EventHandler eventHandler;
- private StoreTableNode store = null;
- private LogicalNode plan = null;
- private List<ScanNode> scan;
-
- private Map<String, Set<FragmentProto>> fragMap;
- private Map<String, Set<FetchImpl>> fetchMap;
-
- private int totalFragmentNum;
-
- private List<ShuffleFileOutput> shuffleFileOutputs;
- private TableStats stats;
- private final boolean isLeafTask;
- private List<IntermediateEntry> intermediateData;
-
- private Map<TaskAttemptId, TaskAttempt> attempts;
- private final int maxAttempts = 3;
- private Integer nextAttempt = -1;
- private TaskAttemptId lastAttemptId;
-
- private TaskAttemptId successfulAttempt;
- private String succeededHost;
- private int succeededHostPort;
- private int succeededPullServerPort;
-
- private int failedAttempts;
- private int finishedAttempts; // finish are total of success, failed and killed
-
- private long launchTime;
- private long finishTime;
-
- private List<DataLocation> dataLocations = Lists.newArrayList();
-
- private static final AttemptKilledTransition ATTEMPT_KILLED_TRANSITION = new AttemptKilledTransition();
-
- private TaskHistory finalTaskHistory;
-
- protected static final StateMachineFactory
- <Task, TaskState, TaskEventType, TaskEvent> stateMachineFactory =
- new StateMachineFactory <Task, TaskState, TaskEventType, TaskEvent>(TaskState.NEW)
-
- // Transitions from NEW state
- .addTransition(TaskState.NEW, TaskState.SCHEDULED,
- TaskEventType.T_SCHEDULE,
- new InitialScheduleTransition())
- .addTransition(TaskState.NEW, TaskState.KILLED,
- TaskEventType.T_KILL,
- new KillNewTaskTransition())
-
- // Transitions from SCHEDULED state
- .addTransition(TaskState.SCHEDULED, TaskState.RUNNING,
- TaskEventType.T_ATTEMPT_LAUNCHED,
- new AttemptLaunchedTransition())
- .addTransition(TaskState.SCHEDULED, TaskState.KILL_WAIT,
- TaskEventType.T_KILL,
- new KillTaskTransition())
-
- // Transitions from RUNNING state
- .addTransition(TaskState.RUNNING, TaskState.RUNNING,
- TaskEventType.T_ATTEMPT_LAUNCHED)
- .addTransition(TaskState.RUNNING, TaskState.SUCCEEDED,
- TaskEventType.T_ATTEMPT_SUCCEEDED,
- new AttemptSucceededTransition())
- .addTransition(TaskState.RUNNING, TaskState.KILL_WAIT,
- TaskEventType.T_KILL,
- new KillTaskTransition())
- .addTransition(TaskState.RUNNING,
- EnumSet.of(TaskState.RUNNING, TaskState.FAILED),
- TaskEventType.T_ATTEMPT_FAILED,
- new AttemptFailedOrRetryTransition())
-
- // Transitions from KILL_WAIT state
- .addTransition(TaskState.KILL_WAIT, TaskState.KILLED,
- TaskEventType.T_ATTEMPT_KILLED,
- ATTEMPT_KILLED_TRANSITION)
- .addTransition(TaskState.KILL_WAIT, TaskState.KILL_WAIT,
- TaskEventType.T_ATTEMPT_LAUNCHED,
- new KillTaskTransition())
- .addTransition(TaskState.KILL_WAIT, TaskState.FAILED,
- TaskEventType.T_ATTEMPT_FAILED,
- new AttemptFailedTransition())
- .addTransition(TaskState.KILL_WAIT, TaskState.KILLED,
- TaskEventType.T_ATTEMPT_SUCCEEDED,
- ATTEMPT_KILLED_TRANSITION)
- // Ignore-able transitions.
- .addTransition(TaskState.KILL_WAIT, TaskState.KILL_WAIT,
- EnumSet.of(
- TaskEventType.T_KILL,
- TaskEventType.T_SCHEDULE))
-
- // Transitions from SUCCEEDED state
- // Ignore-able transitions
- .addTransition(TaskState.SUCCEEDED, TaskState.SUCCEEDED,
- EnumSet.of(TaskEventType.T_KILL,
- TaskEventType.T_ATTEMPT_KILLED, TaskEventType.T_ATTEMPT_SUCCEEDED, TaskEventType.T_ATTEMPT_FAILED))
-
- // Transitions from FAILED state
- // Ignore-able transitions
- .addTransition(TaskState.FAILED, TaskState.FAILED,
- EnumSet.of(TaskEventType.T_KILL,
- TaskEventType.T_ATTEMPT_KILLED, TaskEventType.T_ATTEMPT_SUCCEEDED, TaskEventType.T_ATTEMPT_FAILED))
-
- // Transitions from KILLED state
- .addTransition(TaskState.KILLED, TaskState.KILLED, TaskEventType.T_ATTEMPT_KILLED, new KillTaskTransition())
- // Ignore-able transitions
- .addTransition(TaskState.KILLED, TaskState.KILLED,
- EnumSet.of(
- TaskEventType.T_KILL, TaskEventType.T_ATTEMPT_SUCCEEDED, TaskEventType.T_ATTEMPT_FAILED))
-
- .installTopology();
-
- private final StateMachine<TaskState, TaskEventType, TaskEvent> stateMachine;
-
-
- private final Lock readLock;
- private final Lock writeLock;
- private TaskAttemptScheduleContext scheduleContext;
-
- public Task(Configuration conf, TaskAttemptScheduleContext scheduleContext,
- TaskId id, boolean isLeafTask, EventHandler eventHandler) {
- this.systemConf = conf;
- this.taskId = id;
- this.eventHandler = eventHandler;
- this.isLeafTask = isLeafTask;
- scan = new ArrayList<ScanNode>();
- fetchMap = Maps.newHashMap();
- fragMap = Maps.newHashMap();
- shuffleFileOutputs = new ArrayList<ShuffleFileOutput>();
- attempts = Collections.emptyMap();
- lastAttemptId = null;
- nextAttempt = -1;
- failedAttempts = 0;
-
- ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
- this.readLock = readWriteLock.readLock();
- this.writeLock = readWriteLock.writeLock();
- this.scheduleContext = scheduleContext;
-
- stateMachine = stateMachineFactory.make(this);
- totalFragmentNum = 0;
- }
-
- public boolean isLeafTask() {
- return this.isLeafTask;
- }
-
- public TaskState getState() {
- readLock.lock();
- try {
- return stateMachine.getCurrentState();
- } finally {
- readLock.unlock();
- }
- }
-
- public TaskAttemptState getLastAttemptStatus() {
- TaskAttempt lastAttempt = getLastAttempt();
- if (lastAttempt != null) {
- return lastAttempt.getState();
- } else {
- return TaskAttemptState.TA_ASSIGNED;
- }
- }
-
- public TaskHistory getTaskHistory() {
- if (finalTaskHistory != null) {
- if (finalTaskHistory.getFinishTime() == 0) {
- finalTaskHistory = makeTaskHistory();
- }
- return finalTaskHistory;
- } else {
- return makeTaskHistory();
- }
- }
-
- private TaskHistory makeTaskHistory() {
- TaskHistory taskHistory = new TaskHistory();
-
- TaskAttempt lastAttempt = getLastAttempt();
- if (lastAttempt != null) {
- taskHistory.setId(lastAttempt.getId().toString());
- taskHistory.setState(lastAttempt.getState().toString());
- taskHistory.setProgress(lastAttempt.getProgress());
- }
- taskHistory.setHostAndPort(succeededHost + ":" + succeededHostPort);
- taskHistory.setRetryCount(this.getRetryCount());
- taskHistory.setLaunchTime(launchTime);
- taskHistory.setFinishTime(finishTime);
-
- taskHistory.setNumShuffles(getShuffleOutpuNum());
- if (!getShuffleFileOutputs().isEmpty()) {
- ShuffleFileOutput shuffleFileOutputs = getShuffleFileOutputs().get(0);
- if (taskHistory.getNumShuffles() > 0) {
- taskHistory.setShuffleKey("" + shuffleFileOutputs.getPartId());
- taskHistory.setShuffleFileName(shuffleFileOutputs.getFileName());
- }
- }
-
- List<String> fragmentList = new ArrayList<String>();
- for (FragmentProto eachFragment : getAllFragments()) {
- try {
- Fragment fragment = FragmentConvertor.convert(systemConf, eachFragment);
- fragmentList.add(fragment.toString());
- } catch (Exception e) {
- LOG.error(e.getMessage());
- fragmentList.add("ERROR: " + eachFragment.getStoreType() + "," + eachFragment.getId() + ": " + e.getMessage());
- }
- }
- taskHistory.setFragments(fragmentList.toArray(new String[]{}));
-
- List<String[]> fetchList = new ArrayList<String[]>();
- for (Map.Entry<String, Set<FetchImpl>> e : getFetchMap().entrySet()) {
- for (FetchImpl f : e.getValue()) {
- for (URI uri : f.getSimpleURIs()){
- fetchList.add(new String[] {e.getKey(), uri.toString()});
- }
- }
- }
-
- taskHistory.setFetchs(fetchList.toArray(new String[][]{}));
-
- List<String> dataLocationList = new ArrayList<String>();
- for(DataLocation eachLocation: getDataLocations()) {
- dataLocationList.add(eachLocation.toString());
- }
-
- taskHistory.setDataLocations(dataLocationList.toArray(new String[]{}));
- return taskHistory;
- }
-
- public void setLogicalPlan(LogicalNode plan) {
- this.plan = plan;
-
- LogicalNode node = plan;
- ArrayList<LogicalNode> s = new ArrayList<LogicalNode>();
- s.add(node);
- while (!s.isEmpty()) {
- node = s.remove(s.size()-1);
- if (node instanceof UnaryNode) {
- UnaryNode unary = (UnaryNode) node;
- s.add(s.size(), unary.getChild());
- } else if (node instanceof BinaryNode) {
- BinaryNode binary = (BinaryNode) node;
- s.add(s.size(), binary.getLeftChild());
- s.add(s.size(), binary.getRightChild());
- } else if (node instanceof ScanNode) {
- scan.add((ScanNode)node);
- } else if (node instanceof TableSubQueryNode) {
- s.add(((TableSubQueryNode) node).getSubQuery());
- }
- }
- }
-
- private void addDataLocation(Fragment fragment) {
- String[] hosts = fragment.getHosts();
- int[] diskIds = null;
- if (fragment instanceof FileFragment) {
- diskIds = ((FileFragment)fragment).getDiskIds();
- }
- for (int i = 0; i < hosts.length; i++) {
- dataLocations.add(new DataLocation(hosts[i], diskIds == null ? -1 : diskIds[i]));
- }
- }
-
- public void addFragment(Fragment fragment, boolean useDataLocation) {
- Set<FragmentProto> fragmentProtos;
- if (fragMap.containsKey(fragment.getTableName())) {
- fragmentProtos = fragMap.get(fragment.getTableName());
- } else {
- fragmentProtos = new HashSet<FragmentProto>();
- fragMap.put(fragment.getTableName(), fragmentProtos);
- }
- fragmentProtos.add(fragment.getProto());
- if (useDataLocation) {
- addDataLocation(fragment);
- }
- totalFragmentNum++;
- }
-
- public void addFragments(Collection<Fragment> fragments) {
- for (Fragment eachFragment: fragments) {
- addFragment(eachFragment, false);
- }
- }
-
- public void setFragment(FragmentPair[] fragmentPairs) {
- for (FragmentPair eachFragmentPair : fragmentPairs) {
- this.addFragment(eachFragmentPair.getLeftFragment(), true);
- if (eachFragmentPair.getRightFragment() != null) {
- this.addFragment(eachFragmentPair.getRightFragment(), true);
- }
- }
- }
-
- public List<DataLocation> getDataLocations() {
- return dataLocations;
- }
-
- public String getSucceededHost() {
- return succeededHost;
- }
-
- public void addFetches(String tableId, Collection<FetchImpl> fetches) {
- Set<FetchImpl> fetchSet;
- if (fetchMap.containsKey(tableId)) {
- fetchSet = fetchMap.get(tableId);
- } else {
- fetchSet = Sets.newHashSet();
- }
- fetchSet.addAll(fetches);
- fetchMap.put(tableId, fetchSet);
- }
-
- public void setFetches(Map<String, Set<FetchImpl>> fetches) {
- this.fetchMap.clear();
- this.fetchMap.putAll(fetches);
- }
-
- public Collection<FragmentProto> getAllFragments() {
- Set<FragmentProto> fragmentProtos = new HashSet<FragmentProto>();
- for (Set<FragmentProto> eachFragmentSet : fragMap.values()) {
- fragmentProtos.addAll(eachFragmentSet);
- }
- return fragmentProtos;
- }
-
- public LogicalNode getLogicalPlan() {
- return this.plan;
- }
-
- public TaskId getId() {
- return taskId;
- }
-
- public Collection<FetchImpl> getFetchHosts(String tableId) {
- return fetchMap.get(tableId);
- }
-
- public Collection<Set<FetchImpl>> getFetches() {
- return fetchMap.values();
- }
-
- public Map<String, Set<FetchImpl>> getFetchMap() {
- return fetchMap;
- }
-
- public Collection<FetchImpl> getFetch(ScanNode scan) {
- return this.fetchMap.get(scan.getTableName());
- }
-
- public ScanNode[] getScanNodes() {
- return this.scan.toArray(new ScanNode[scan.size()]);
- }
-
- @Override
- public String toString() {
- StringBuilder builder = new StringBuilder();
- builder.append(plan.getType() + " \n");
- for (Entry<String, Set<FragmentProto>> e : fragMap.entrySet()) {
- builder.append(e.getKey()).append(" : ");
- for (FragmentProto fragment : e.getValue()) {
- builder.append(fragment).append(", ");
- }
- }
- for (Entry<String, Set<FetchImpl>> e : fetchMap.entrySet()) {
- builder.append(e.getKey()).append(" : ");
- for (FetchImpl t : e.getValue()) {
- for (URI uri : t.getURIs()){
- builder.append(uri).append(" ");
- }
- }
- }
-
- return builder.toString();
- }
-
- public void setStats(TableStats stats) {
- this.stats = stats;
- }
-
- public void setShuffleFileOutputs(List<ShuffleFileOutput> partitions) {
- this.shuffleFileOutputs = Collections.unmodifiableList(partitions);
- }
-
- public TableStats getStats() {
- return this.stats;
- }
-
- public List<ShuffleFileOutput> getShuffleFileOutputs() {
- return this.shuffleFileOutputs;
- }
-
- public int getShuffleOutpuNum() {
- return this.shuffleFileOutputs.size();
- }
-
- public TaskAttempt newAttempt() {
- TaskAttempt attempt = new TaskAttempt(scheduleContext,
- QueryIdFactory.newTaskAttemptId(this.getId(), ++nextAttempt),
- this, eventHandler);
- lastAttemptId = attempt.getId();
- return attempt;
- }
-
- public TaskAttempt getAttempt(TaskAttemptId attemptId) {
- return attempts.get(attemptId);
- }
-
- public TaskAttempt getAttempt(int attempt) {
- return this.attempts.get(QueryIdFactory.newTaskAttemptId(this.getId(), attempt));
- }
-
- public TaskAttempt getLastAttempt() {
- return getAttempt(this.lastAttemptId);
- }
-
- public TaskAttempt getSuccessfulAttempt() {
- readLock.lock();
- try {
- if (null == successfulAttempt) {
- return null;
- }
- return attempts.get(successfulAttempt);
- } finally {
- readLock.unlock();
- }
- }
-
- public int getRetryCount () {
- return this.nextAttempt;
- }
-
- public int getTotalFragmentNum() {
- return totalFragmentNum;
- }
-
- private static class InitialScheduleTransition implements
- SingleArcTransition<Task, TaskEvent> {
-
- @Override
- public void transition(Task task, TaskEvent taskEvent) {
- task.addAndScheduleAttempt();
- }
- }
-
- public long getLaunchTime() {
- return launchTime;
- }
-
- public long getFinishTime() {
- return finishTime;
- }
-
- @VisibleForTesting
- public void setLaunchTime(long launchTime) {
- this.launchTime = launchTime;
- }
-
- @VisibleForTesting
- public void setFinishTime(long finishTime) {
- this.finishTime = finishTime;
- }
-
- public long getRunningTime() {
- if(finishTime > 0) {
- return finishTime - launchTime;
- } else {
- return System.currentTimeMillis() - launchTime;
- }
- }
-
- // This is always called in the Write Lock
- private void addAndScheduleAttempt() {
- // Create new task attempt
- TaskAttempt attempt = newAttempt();
- if (LOG.isDebugEnabled()) {
- LOG.debug("Created attempt " + attempt.getId());
- }
- switch (attempts.size()) {
- case 0:
- attempts = Collections.singletonMap(attempt.getId(), attempt);
- break;
-
- case 1:
- Map<TaskAttemptId, TaskAttempt> newAttempts
- = new LinkedHashMap<TaskAttemptId, TaskAttempt>(3);
- newAttempts.putAll(attempts);
- attempts = newAttempts;
- attempts.put(attempt.getId(), attempt);
- break;
-
- default:
- attempts.put(attempt.getId(), attempt);
- break;
- }
-
- if (failedAttempts > 0) {
- eventHandler.handle(new TaskAttemptScheduleEvent(systemConf, attempt.getId(),
- TaskAttemptEventType.TA_RESCHEDULE));
- } else {
- eventHandler.handle(new TaskAttemptScheduleEvent(systemConf, attempt.getId(),
- TaskAttemptEventType.TA_SCHEDULE));
- }
- }
-
- private void finishTask() {
- this.finishTime = System.currentTimeMillis();
- finalTaskHistory = makeTaskHistory();
- }
-
- private static class KillNewTaskTransition implements SingleArcTransition<Task, TaskEvent> {
-
- @Override
- public void transition(Task task, TaskEvent taskEvent) {
- task.eventHandler.handle(new StageTaskEvent(task.getId(), TaskState.KILLED));
- }
- }
-
- private static class KillTaskTransition implements SingleArcTransition<Task, TaskEvent> {
-
- @Override
- public void transition(Task task, TaskEvent taskEvent) {
- task.finishTask();
- task.eventHandler.handle(new TaskAttemptEvent(task.lastAttemptId, TaskAttemptEventType.TA_KILL));
- }
- }
-
- private static class AttemptKilledTransition implements SingleArcTransition<Task, TaskEvent>{
-
- @Override
- public void transition(Task task, TaskEvent event) {
- task.finishTask();
- task.eventHandler.handle(new StageTaskEvent(task.getId(), TaskState.KILLED));
- }
- }
-
- private static class AttemptSucceededTransition
- implements SingleArcTransition<Task, TaskEvent>{
-
- @Override
- public void transition(Task task,
- TaskEvent event) {
- TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) event;
- TaskAttempt attempt = task.attempts.get(attemptEvent.getTaskAttemptId());
-
- task.successfulAttempt = attemptEvent.getTaskAttemptId();
- task.succeededHost = attempt.getWorkerConnectionInfo().getHost();
- task.succeededHostPort = attempt.getWorkerConnectionInfo().getPeerRpcPort();
- task.succeededPullServerPort = attempt.getWorkerConnectionInfo().getPullServerPort();
-
- task.finishTask();
- task.eventHandler.handle(new StageTaskEvent(event.getTaskId(), TaskState.SUCCEEDED));
- }
- }
-
- private static class AttemptLaunchedTransition implements SingleArcTransition<Task, TaskEvent> {
- @Override
- public void transition(Task task,
- TaskEvent event) {
- TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) event;
- TaskAttempt attempt = task.attempts.get(attemptEvent.getTaskAttemptId());
- task.launchTime = System.currentTimeMillis();
- task.succeededHost = attempt.getWorkerConnectionInfo().getHost();
- task.succeededHostPort = attempt.getWorkerConnectionInfo().getPeerRpcPort();
- }
- }
-
- private static class AttemptFailedTransition implements SingleArcTransition<Task, TaskEvent> {
- @Override
- public void transition(Task task, TaskEvent event) {
- TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) event;
- LOG.info("=============================================================");
- LOG.info(">>> Task Failed: " + attemptEvent.getTaskAttemptId() + " <<<");
- LOG.info("=============================================================");
- task.failedAttempts++;
- task.finishedAttempts++;
-
- task.finishTask();
- task.eventHandler.handle(new StageTaskEvent(task.getId(), TaskState.FAILED));
- }
- }
-
- private static class AttemptFailedOrRetryTransition implements
- MultipleArcTransition<Task, TaskEvent, TaskState> {
-
- @Override
- public TaskState transition(Task task, TaskEvent taskEvent) {
- TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) taskEvent;
- task.failedAttempts++;
- task.finishedAttempts++;
- boolean retry = task.failedAttempts < task.maxAttempts;
-
- LOG.info("====================================================================================");
- LOG.info(">>> Task Failed: " + attemptEvent.getTaskAttemptId() + ", " +
- "retry:" + retry + ", attempts:" + task.failedAttempts + " <<<");
- LOG.info("====================================================================================");
-
- if (retry) {
- if (task.successfulAttempt == null) {
- task.addAndScheduleAttempt();
- }
- } else {
- task.finishTask();
- task.eventHandler.handle(new StageTaskEvent(task.getId(), TaskState.FAILED));
- return TaskState.FAILED;
- }
-
- return task.getState();
- }
- }
-
- @Override
- public void handle(TaskEvent event) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Processing " + event.getTaskId() + " of type "
- + event.getType());
- }
-
- try {
- writeLock.lock();
- TaskState oldState = getState();
- try {
- stateMachine.doTransition(event.getType(), event);
- } catch (InvalidStateTransitonException e) {
- LOG.error("Can't handle this event at current state"
- + ", eventType:" + event.getType().name()
- + ", oldState:" + oldState.name()
- + ", nextState:" + getState().name()
- , e);
- eventHandler.handle(new QueryEvent(TajoIdUtils.parseQueryId(getId().toString()),
- QueryEventType.INTERNAL_ERROR));
- }
-
- //notify the eventhandler of state change
- if (LOG.isDebugEnabled()) {
- if (oldState != getState()) {
- LOG.debug(taskId + " Task Transitioned from " + oldState + " to "
- + getState());
- }
- }
- }
-
- finally {
- writeLock.unlock();
- }
- }
-
- public void setIntermediateData(Collection<IntermediateEntry> partitions) {
- this.intermediateData = new ArrayList<IntermediateEntry>(partitions);
- }
-
- public List<IntermediateEntry> getIntermediateData() {
- return this.intermediateData;
- }
-
- public static class PullHost implements Cloneable {
- String host;
- int port;
- int hashCode;
-
- public PullHost(String pullServerAddr, int pullServerPort){
- this.host = pullServerAddr;
- this.port = pullServerPort;
- this.hashCode = Objects.hashCode(host, port);
- }
- public String getHost() {
- return host;
- }
-
- public int getPort() {
- return this.port;
- }
-
- public String getPullAddress() {
- return host + ":" + port;
- }
-
- @Override
- public int hashCode() {
- return hashCode;
- }
-
- @Override
- public boolean equals(Object obj) {
- if (obj instanceof PullHost) {
- PullHost other = (PullHost) obj;
- return host.equals(other.host) && port == other.port;
- }
-
- return false;
- }
-
- @Override
- public PullHost clone() throws CloneNotSupportedException {
- PullHost newPullHost = (PullHost) super.clone();
- newPullHost.host = host;
- newPullHost.port = port;
- newPullHost.hashCode = Objects.hashCode(newPullHost.host, newPullHost.port);
- return newPullHost;
- }
-
- @Override
- public String toString() {
- return host + ":" + port;
- }
- }
-
- public static class IntermediateEntry {
- ExecutionBlockId ebId;
- int taskId;
- int attemptId;
- int partId;
- PullHost host;
- long volume;
- List<Pair<Long, Integer>> pages;
- List<Pair<Long, Pair<Integer, Integer>>> failureRowNums;
-
- public IntermediateEntry(IntermediateEntryProto proto) {
- this.ebId = new ExecutionBlockId(proto.getEbId());
- this.taskId = proto.getTaskId();
- this.attemptId = proto.getAttemptId();
- this.partId = proto.getPartId();
-
- String[] pullHost = proto.getHost().split(":");
- this.host = new PullHost(pullHost[0], Integer.parseInt(pullHost[1]));
- this.volume = proto.getVolume();
-
- failureRowNums = new ArrayList<Pair<Long, Pair<Integer, Integer>>>();
- for (FailureIntermediateProto eachFailure: proto.getFailuresList()) {
-
- failureRowNums.add(new Pair(eachFailure.getPagePos(),
- new Pair(eachFailure.getStartRowNum(), eachFailure.getEndRowNum())));
- }
-
- pages = new ArrayList<Pair<Long, Integer>>();
- for (IntermediateEntryProto.PageProto eachPage: proto.getPagesList()) {
- pages.add(new Pair(eachPage.getPos(), eachPage.getLength()));
- }
- }
-
- public IntermediateEntry(int taskId, int attemptId, int partId, PullHost host) {
- this.taskId = taskId;
- this.attemptId = attemptId;
- this.partId = partId;
- this.host = host;
- }
-
- public IntermediateEntry(int taskId, int attemptId, int partId, PullHost host, long volume) {
- this.taskId = taskId;
- this.attemptId = attemptId;
- this.partId = partId;
- this.host = host;
- this.volume = volume;
- }
-
- public ExecutionBlockId getEbId() {
- return ebId;
- }
-
- public void setEbId(ExecutionBlockId ebId) {
- this.ebId = ebId;
- }
-
- public int getTaskId() {
- return this.taskId;
- }
-
- public int getAttemptId() {
- return this.attemptId;
- }
-
- public int getPartId() {
- return this.partId;
- }
-
- public PullHost getPullHost() {
- return this.host;
- }
-
- public long getVolume() {
- return this.volume;
- }
-
- public long setVolume(long volume) {
- return this.volume = volume;
- }
-
- public List<Pair<Long, Integer>> getPages() {
- return pages;
- }
-
- public void setPages(List<Pair<Long, Integer>> pages) {
- this.pages = pages;
- }
-
- public List<Pair<Long, Pair<Integer, Integer>>> getFailureRowNums() {
- return failureRowNums;
- }
-
- @Override
- public int hashCode() {
- return Objects.hashCode(ebId, taskId, partId, attemptId, host);
- }
-
- public List<Pair<Long, Long>> split(long firstSplitVolume, long splitVolume) {
- List<Pair<Long, Long>> splits = new ArrayList<Pair<Long, Long>>();
-
- if (pages == null || pages.isEmpty()) {
- return splits;
- }
- int pageSize = pages.size();
-
- long currentOffset = -1;
- long currentBytes = 0;
-
- long realSplitVolume = firstSplitVolume > 0 ? firstSplitVolume : splitVolume;
- for (int i = 0; i < pageSize; i++) {
- Pair<Long, Integer> eachPage = pages.get(i);
- if (currentOffset == -1) {
- currentOffset = eachPage.getFirst();
- }
- if (currentBytes > 0 && currentBytes + eachPage.getSecond() >= realSplitVolume) {
- splits.add(new Pair(currentOffset, currentBytes));
- currentOffset = eachPage.getFirst();
- currentBytes = 0;
- realSplitVolume = splitVolume;
- }
-
- currentBytes += eachPage.getSecond();
- }
-
- //add last
- if (currentBytes > 0) {
- splits.add(new Pair(currentOffset, currentBytes));
- }
- return splits;
- }
- }
-}
[04/16] tajo git commit: TAJO-1288: Refactoring
org.apache.tajo.master package.
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
new file mode 100644
index 0000000..34c58d4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Stage.java
@@ -0,0 +1,1342 @@
+/**
+ * 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.tajo.querymaster;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.state.*;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.tajo.*;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.statistics.ColumnStats;
+import org.apache.tajo.catalog.statistics.StatisticsUtil;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
+import org.apache.tajo.engine.planner.enforce.Enforcer;
+import org.apache.tajo.engine.planner.global.DataChannel;
+import org.apache.tajo.engine.planner.global.ExecutionBlock;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.MultipleAggregationStage;
+import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty;
+import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto;
+import org.apache.tajo.master.*;
+import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext;
+import org.apache.tajo.querymaster.Task.IntermediateEntry;
+import org.apache.tajo.master.container.TajoContainer;
+import org.apache.tajo.master.container.TajoContainerId;
+import org.apache.tajo.storage.FileStorageManager;
+import org.apache.tajo.plan.util.PlannerUtil;
+import org.apache.tajo.plan.logical.*;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.unit.StorageUnit;
+import org.apache.tajo.util.KeyValueSet;
+import org.apache.tajo.util.history.TaskHistory;
+import org.apache.tajo.util.history.StageHistory;
+import org.apache.tajo.worker.FetchImpl;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static org.apache.tajo.conf.TajoConf.ConfVars;
+import static org.apache.tajo.plan.serder.PlanProto.ShuffleType;
+
+
+/**
+ * Stage plays a role in controlling an ExecutionBlock and is a finite state machine.
+ */
+public class Stage implements EventHandler<StageEvent> {
+
+ private static final Log LOG = LogFactory.getLog(Stage.class);
+
+ private MasterPlan masterPlan;
+ private ExecutionBlock block;
+ private int priority;
+ private Schema schema;
+ private TableMeta meta;
+ private TableStats resultStatistics;
+ private TableStats inputStatistics;
+ private EventHandler<Event> eventHandler;
+ private AbstractTaskScheduler taskScheduler;
+ private QueryMasterTask.QueryMasterTaskContext context;
+ private final List<String> diagnostics = new ArrayList<String>();
+ private StageState stageState;
+
+ private long startTime;
+ private long finishTime;
+
+ volatile Map<TaskId, Task> tasks = new ConcurrentHashMap<TaskId, Task>();
+ volatile Map<TajoContainerId, TajoContainer> containers = new ConcurrentHashMap<TajoContainerId,
+ TajoContainer>();
+
+ private static final DiagnosticsUpdateTransition DIAGNOSTIC_UPDATE_TRANSITION = new DiagnosticsUpdateTransition();
+ private static final InternalErrorTransition INTERNAL_ERROR_TRANSITION = new InternalErrorTransition();
+ private static final ContainerLaunchTransition CONTAINER_LAUNCH_TRANSITION = new ContainerLaunchTransition();
+ private static final TaskCompletedTransition TASK_COMPLETED_TRANSITION = new TaskCompletedTransition();
+ private static final AllocatedContainersCancelTransition CONTAINERS_CANCEL_TRANSITION =
+ new AllocatedContainersCancelTransition();
+ private static final StageCompleteTransition STAGE_COMPLETED_TRANSITION = new StageCompleteTransition();
+ private StateMachine<StageState, StageEventType, StageEvent> stateMachine;
+
+ protected static final StateMachineFactory<Stage, StageState,
+ StageEventType, StageEvent> stateMachineFactory =
+ new StateMachineFactory <Stage, StageState,
+ StageEventType, StageEvent> (StageState.NEW)
+
+ // Transitions from NEW state
+ .addTransition(StageState.NEW,
+ EnumSet.of(StageState.INITED, StageState.ERROR, StageState.SUCCEEDED),
+ StageEventType.SQ_INIT,
+ new InitAndRequestContainer())
+ .addTransition(StageState.NEW, StageState.NEW,
+ StageEventType.SQ_DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(StageState.NEW, StageState.KILLED,
+ StageEventType.SQ_KILL)
+ .addTransition(StageState.NEW, StageState.ERROR,
+ StageEventType.SQ_INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+
+ // Transitions from INITED state
+ .addTransition(StageState.INITED, StageState.RUNNING,
+ StageEventType.SQ_CONTAINER_ALLOCATED,
+ CONTAINER_LAUNCH_TRANSITION)
+ .addTransition(StageState.INITED, StageState.INITED,
+ StageEventType.SQ_DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(StageState.INITED, StageState.KILL_WAIT,
+ StageEventType.SQ_KILL, new KillTasksTransition())
+ .addTransition(StageState.INITED, StageState.ERROR,
+ StageEventType.SQ_INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+
+ // Transitions from RUNNING state
+ .addTransition(StageState.RUNNING, StageState.RUNNING,
+ StageEventType.SQ_CONTAINER_ALLOCATED,
+ CONTAINER_LAUNCH_TRANSITION)
+ .addTransition(StageState.RUNNING, StageState.RUNNING,
+ StageEventType.SQ_TASK_COMPLETED,
+ TASK_COMPLETED_TRANSITION)
+ .addTransition(StageState.RUNNING,
+ EnumSet.of(StageState.SUCCEEDED, StageState.FAILED),
+ StageEventType.SQ_STAGE_COMPLETED,
+ STAGE_COMPLETED_TRANSITION)
+ .addTransition(StageState.RUNNING, StageState.RUNNING,
+ StageEventType.SQ_FAILED,
+ TASK_COMPLETED_TRANSITION)
+ .addTransition(StageState.RUNNING, StageState.RUNNING,
+ StageEventType.SQ_DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(StageState.RUNNING, StageState.KILL_WAIT,
+ StageEventType.SQ_KILL,
+ new KillTasksTransition())
+ .addTransition(StageState.RUNNING, StageState.ERROR,
+ StageEventType.SQ_INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+ // Ignore-able Transition
+ .addTransition(StageState.RUNNING, StageState.RUNNING,
+ StageEventType.SQ_START)
+
+ // Transitions from KILL_WAIT state
+ .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
+ StageEventType.SQ_CONTAINER_ALLOCATED,
+ CONTAINERS_CANCEL_TRANSITION)
+ .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
+ EnumSet.of(StageEventType.SQ_KILL), new KillTasksTransition())
+ .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
+ StageEventType.SQ_TASK_COMPLETED,
+ TASK_COMPLETED_TRANSITION)
+ .addTransition(StageState.KILL_WAIT,
+ EnumSet.of(StageState.SUCCEEDED, StageState.FAILED, StageState.KILLED),
+ StageEventType.SQ_STAGE_COMPLETED,
+ STAGE_COMPLETED_TRANSITION)
+ .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
+ StageEventType.SQ_DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(StageState.KILL_WAIT, StageState.KILL_WAIT,
+ StageEventType.SQ_FAILED,
+ TASK_COMPLETED_TRANSITION)
+ .addTransition(StageState.KILL_WAIT, StageState.ERROR,
+ StageEventType.SQ_INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+
+ // Transitions from SUCCEEDED state
+ .addTransition(StageState.SUCCEEDED, StageState.SUCCEEDED,
+ StageEventType.SQ_CONTAINER_ALLOCATED,
+ CONTAINERS_CANCEL_TRANSITION)
+ .addTransition(StageState.SUCCEEDED, StageState.SUCCEEDED,
+ StageEventType.SQ_DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(StageState.SUCCEEDED, StageState.ERROR,
+ StageEventType.SQ_INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+ // Ignore-able events
+ .addTransition(StageState.SUCCEEDED, StageState.SUCCEEDED,
+ EnumSet.of(
+ StageEventType.SQ_START,
+ StageEventType.SQ_KILL,
+ StageEventType.SQ_CONTAINER_ALLOCATED))
+
+ // Transitions from KILLED state
+ .addTransition(StageState.KILLED, StageState.KILLED,
+ StageEventType.SQ_CONTAINER_ALLOCATED,
+ CONTAINERS_CANCEL_TRANSITION)
+ .addTransition(StageState.KILLED, StageState.KILLED,
+ StageEventType.SQ_DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(StageState.KILLED, StageState.ERROR,
+ StageEventType.SQ_INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+ // Ignore-able transitions
+ .addTransition(StageState.KILLED, StageState.KILLED,
+ EnumSet.of(
+ StageEventType.SQ_START,
+ StageEventType.SQ_KILL,
+ StageEventType.SQ_CONTAINER_ALLOCATED,
+ StageEventType.SQ_FAILED))
+
+ // Transitions from FAILED state
+ .addTransition(StageState.FAILED, StageState.FAILED,
+ StageEventType.SQ_CONTAINER_ALLOCATED,
+ CONTAINERS_CANCEL_TRANSITION)
+ .addTransition(StageState.FAILED, StageState.FAILED,
+ StageEventType.SQ_DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(StageState.FAILED, StageState.ERROR,
+ StageEventType.SQ_INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+ // Ignore-able transitions
+ .addTransition(StageState.FAILED, StageState.FAILED,
+ EnumSet.of(
+ StageEventType.SQ_START,
+ StageEventType.SQ_KILL,
+ StageEventType.SQ_CONTAINER_ALLOCATED,
+ StageEventType.SQ_FAILED))
+
+ // Transitions from ERROR state
+ .addTransition(StageState.ERROR, StageState.ERROR,
+ StageEventType.SQ_CONTAINER_ALLOCATED,
+ CONTAINERS_CANCEL_TRANSITION)
+ .addTransition(StageState.ERROR, StageState.ERROR,
+ StageEventType.SQ_DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ // Ignore-able transitions
+ .addTransition(StageState.ERROR, StageState.ERROR,
+ EnumSet.of(
+ StageEventType.SQ_START,
+ StageEventType.SQ_KILL,
+ StageEventType.SQ_FAILED,
+ StageEventType.SQ_INTERNAL_ERROR,
+ StageEventType.SQ_STAGE_COMPLETED))
+
+ .installTopology();
+
+ private final Lock readLock;
+ private final Lock writeLock;
+
+ private int totalScheduledObjectsCount;
+ private int succeededObjectCount = 0;
+ private int completedTaskCount = 0;
+ private int succeededTaskCount = 0;
+ private int killedObjectCount = 0;
+ private int failedObjectCount = 0;
+ private TaskSchedulerContext schedulerContext;
+ private List<IntermediateEntry> hashShuffleIntermediateEntries = new ArrayList<IntermediateEntry>();
+ private AtomicInteger completeReportReceived = new AtomicInteger(0);
+ private StageHistory finalStageHistory;
+
+ public Stage(QueryMasterTask.QueryMasterTaskContext context, MasterPlan masterPlan, ExecutionBlock block) {
+ this.context = context;
+ this.masterPlan = masterPlan;
+ this.block = block;
+ this.eventHandler = context.getEventHandler();
+
+ ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+ this.readLock = readWriteLock.readLock();
+ this.writeLock = readWriteLock.writeLock();
+ stateMachine = stateMachineFactory.make(this);
+ stageState = stateMachine.getCurrentState();
+ }
+
+ public static boolean isRunningState(StageState state) {
+ return state == StageState.INITED || state == StageState.NEW || state == StageState.RUNNING;
+ }
+
+ public QueryMasterTask.QueryMasterTaskContext getContext() {
+ return context;
+ }
+
+ public MasterPlan getMasterPlan() {
+ return masterPlan;
+ }
+
+ public DataChannel getDataChannel() {
+ return masterPlan.getOutgoingChannels(getId()).iterator().next();
+ }
+
+ public EventHandler<Event> getEventHandler() {
+ return eventHandler;
+ }
+
+ public AbstractTaskScheduler getTaskScheduler() {
+ return taskScheduler;
+ }
+
+ public void setStartTime() {
+ startTime = context.getClock().getTime();
+ }
+
+ @SuppressWarnings("UnusedDeclaration")
+ public long getStartTime() {
+ return this.startTime;
+ }
+
+ public void setFinishTime() {
+ finishTime = context.getClock().getTime();
+ }
+
+ @SuppressWarnings("UnusedDeclaration")
+ public long getFinishTime() {
+ return this.finishTime;
+ }
+
+ public float getTaskProgress() {
+ readLock.lock();
+ try {
+ if (getState() == StageState.NEW) {
+ return 0;
+ } else {
+ return (float)(succeededObjectCount) / (float)totalScheduledObjectsCount;
+ }
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ public float getProgress() {
+ List<Task> tempTasks = null;
+ readLock.lock();
+ try {
+ if (getState() == StageState.NEW) {
+ return 0.0f;
+ } else {
+ tempTasks = new ArrayList<Task>(tasks.values());
+ }
+ } finally {
+ readLock.unlock();
+ }
+
+ float totalProgress = 0.0f;
+ for (Task eachTask : tempTasks) {
+ if (eachTask.getLastAttempt() != null) {
+ totalProgress += eachTask.getLastAttempt().getProgress();
+ }
+ }
+
+ if (totalProgress > 0.0f) {
+ return (float) Math.floor((totalProgress / (float) Math.max(tempTasks.size(), 1)) * 1000.0f) / 1000.0f;
+ } else {
+ return 0.0f;
+ }
+ }
+
+ public int getSucceededObjectCount() {
+ return succeededObjectCount;
+ }
+
+ public int getTotalScheduledObjectsCount() {
+ return totalScheduledObjectsCount;
+ }
+
+ public ExecutionBlock getBlock() {
+ return block;
+ }
+
+ public void addTask(Task task) {
+ tasks.put(task.getId(), task);
+ }
+
+ public StageHistory getStageHistory() {
+ if (finalStageHistory != null) {
+ if (finalStageHistory.getFinishTime() == 0) {
+ finalStageHistory = makeStageHistory();
+ finalStageHistory.setTasks(makeTaskHistories());
+ }
+ return finalStageHistory;
+ } else {
+ return makeStageHistory();
+ }
+ }
+
+ private List<TaskHistory> makeTaskHistories() {
+ List<TaskHistory> taskHistories = new ArrayList<TaskHistory>();
+
+ for(Task eachTask : getTasks()) {
+ taskHistories.add(eachTask.getTaskHistory());
+ }
+
+ return taskHistories;
+ }
+
+ private StageHistory makeStageHistory() {
+ StageHistory stageHistory = new StageHistory();
+
+ stageHistory.setExecutionBlockId(getId().toString());
+ stageHistory.setPlan(PlannerUtil.buildExplainString(block.getPlan()));
+ stageHistory.setState(getState().toString());
+ stageHistory.setStartTime(startTime);
+ stageHistory.setFinishTime(finishTime);
+ stageHistory.setSucceededObjectCount(succeededObjectCount);
+ stageHistory.setKilledObjectCount(killedObjectCount);
+ stageHistory.setFailedObjectCount(failedObjectCount);
+ stageHistory.setTotalScheduledObjectsCount(totalScheduledObjectsCount);
+ stageHistory.setHostLocalAssigned(getTaskScheduler().getHostLocalAssigned());
+ stageHistory.setRackLocalAssigned(getTaskScheduler().getRackLocalAssigned());
+
+ long totalInputBytes = 0;
+ long totalReadBytes = 0;
+ long totalReadRows = 0;
+ long totalWriteBytes = 0;
+ long totalWriteRows = 0;
+ int numShuffles = 0;
+ for(Task eachTask : getTasks()) {
+ numShuffles = eachTask.getShuffleOutpuNum();
+ if (eachTask.getLastAttempt() != null) {
+ TableStats inputStats = eachTask.getLastAttempt().getInputStats();
+ if (inputStats != null) {
+ totalInputBytes += inputStats.getNumBytes();
+ totalReadBytes += inputStats.getReadBytes();
+ totalReadRows += inputStats.getNumRows();
+ }
+ TableStats outputStats = eachTask.getLastAttempt().getResultStats();
+ if (outputStats != null) {
+ totalWriteBytes += outputStats.getNumBytes();
+ totalWriteRows += outputStats.getNumRows();
+ }
+ }
+ }
+
+ stageHistory.setTotalInputBytes(totalInputBytes);
+ stageHistory.setTotalReadBytes(totalReadBytes);
+ stageHistory.setTotalReadRows(totalReadRows);
+ stageHistory.setTotalWriteBytes(totalWriteBytes);
+ stageHistory.setTotalWriteRows(totalWriteRows);
+ stageHistory.setNumShuffles(numShuffles);
+ stageHistory.setProgress(getProgress());
+ return stageHistory;
+ }
+
+ /**
+ * It finalizes this stage. It is only invoked when the stage is succeeded.
+ */
+ public void complete() {
+ cleanup();
+ finalizeStats();
+ setFinishTime();
+ eventHandler.handle(new StageCompletedEvent(getId(), StageState.SUCCEEDED));
+ }
+
+ /**
+ * It finalizes this stage. Unlike {@link Stage#complete()},
+ * it is invoked when a stage is abnormally finished.
+ *
+ * @param finalState The final stage state
+ */
+ public void abort(StageState finalState) {
+ // TODO -
+ // - committer.abortStage(...)
+ // - record Stage Finish Time
+ // - CleanUp Tasks
+ // - Record History
+ cleanup();
+ setFinishTime();
+ eventHandler.handle(new StageCompletedEvent(getId(), finalState));
+ }
+
+ public StateMachine<StageState, StageEventType, StageEvent> getStateMachine() {
+ return this.stateMachine;
+ }
+
+ public void setPriority(int priority) {
+ this.priority = priority;
+ }
+
+
+ public int getPriority() {
+ return this.priority;
+ }
+
+ public ExecutionBlockId getId() {
+ return block.getId();
+ }
+
+ public Task[] getTasks() {
+ return tasks.values().toArray(new Task[tasks.size()]);
+ }
+
+ public Task getTask(TaskId qid) {
+ return tasks.get(qid);
+ }
+
+ public Schema getSchema() {
+ return schema;
+ }
+
+ public TableMeta getTableMeta() {
+ return meta;
+ }
+
+ public TableStats getResultStats() {
+ return resultStatistics;
+ }
+
+ public TableStats getInputStats() {
+ return inputStatistics;
+ }
+
+ public List<String> getDiagnostics() {
+ readLock.lock();
+ try {
+ return diagnostics;
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ protected void addDiagnostic(String diag) {
+ diagnostics.add(diag);
+ }
+
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append(this.getId());
+ return sb.toString();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o instanceof Stage) {
+ Stage other = (Stage)o;
+ return getId().equals(other.getId());
+ }
+ return false;
+ }
+
+ @Override
+ public int hashCode() {
+ return getId().hashCode();
+ }
+
+ public int compareTo(Stage other) {
+ return getId().compareTo(other.getId());
+ }
+
+ public StageState getSynchronizedState() {
+ readLock.lock();
+ try {
+ return stateMachine.getCurrentState();
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ /* non-blocking call for client API */
+ public StageState getState() {
+ return stageState;
+ }
+
+ public static TableStats[] computeStatFromUnionBlock(Stage stage) {
+ TableStats[] stat = new TableStats[]{new TableStats(), new TableStats()};
+ long[] avgRows = new long[]{0, 0};
+ long[] numBytes = new long[]{0, 0};
+ long[] readBytes = new long[]{0, 0};
+ long[] numRows = new long[]{0, 0};
+ int[] numBlocks = new int[]{0, 0};
+ int[] numOutputs = new int[]{0, 0};
+
+ List<ColumnStats> columnStatses = Lists.newArrayList();
+
+ MasterPlan masterPlan = stage.getMasterPlan();
+ Iterator<ExecutionBlock> it = masterPlan.getChilds(stage.getBlock()).iterator();
+ while (it.hasNext()) {
+ ExecutionBlock block = it.next();
+ Stage childStage = stage.context.getStage(block.getId());
+ TableStats[] childStatArray = new TableStats[]{
+ childStage.getInputStats(), childStage.getResultStats()
+ };
+ for (int i = 0; i < 2; i++) {
+ if (childStatArray[i] == null) {
+ continue;
+ }
+ avgRows[i] += childStatArray[i].getAvgRows();
+ numBlocks[i] += childStatArray[i].getNumBlocks();
+ numBytes[i] += childStatArray[i].getNumBytes();
+ readBytes[i] += childStatArray[i].getReadBytes();
+ numOutputs[i] += childStatArray[i].getNumShuffleOutputs();
+ numRows[i] += childStatArray[i].getNumRows();
+ }
+ columnStatses.addAll(childStatArray[1].getColumnStats());
+ }
+
+ for (int i = 0; i < 2; i++) {
+ stat[i].setNumBlocks(numBlocks[i]);
+ stat[i].setNumBytes(numBytes[i]);
+ stat[i].setReadBytes(readBytes[i]);
+ stat[i].setNumShuffleOutputs(numOutputs[i]);
+ stat[i].setNumRows(numRows[i]);
+ stat[i].setAvgRows(avgRows[i]);
+ }
+ stat[1].setColumnStats(columnStatses);
+
+ return stat;
+ }
+
+ private TableStats[] computeStatFromTasks() {
+ List<TableStats> inputStatsList = Lists.newArrayList();
+ List<TableStats> resultStatsList = Lists.newArrayList();
+ for (Task unit : getTasks()) {
+ resultStatsList.add(unit.getStats());
+ if (unit.getLastAttempt().getInputStats() != null) {
+ inputStatsList.add(unit.getLastAttempt().getInputStats());
+ }
+ }
+ TableStats inputStats = StatisticsUtil.aggregateTableStat(inputStatsList);
+ TableStats resultStats = StatisticsUtil.aggregateTableStat(resultStatsList);
+ return new TableStats[]{inputStats, resultStats};
+ }
+
+ private void stopScheduler() {
+ // If there are launched TaskRunners, send the 'shouldDie' message to all r
+ // via received task requests.
+ if (taskScheduler != null) {
+ taskScheduler.stop();
+ }
+ }
+
+ private void releaseContainers() {
+ // If there are still live TaskRunners, try to kill the containers.
+ eventHandler.handle(new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP, getId(), containers.values()));
+ }
+
+ /**
+ * It computes all stats and sets the intermediate result.
+ */
+ private void finalizeStats() {
+ TableStats[] statsArray;
+ if (block.hasUnion()) {
+ statsArray = computeStatFromUnionBlock(this);
+ } else {
+ statsArray = computeStatFromTasks();
+ }
+
+ DataChannel channel = masterPlan.getOutgoingChannels(getId()).get(0);
+
+ // if store plan (i.e., CREATE or INSERT OVERWRITE)
+ StoreType storeType = PlannerUtil.getStoreType(masterPlan.getLogicalPlan());
+ if (storeType == null) {
+ // get default or store type
+ storeType = StoreType.CSV;
+ }
+
+ schema = channel.getSchema();
+ meta = CatalogUtil.newTableMeta(storeType, new KeyValueSet());
+ inputStatistics = statsArray[0];
+ resultStatistics = statsArray[1];
+ }
+
+ @Override
+ public void handle(StageEvent event) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Processing " + event.getStageId() + " of type " + event.getType() + ", preState="
+ + getSynchronizedState());
+ }
+
+ try {
+ writeLock.lock();
+ StageState oldState = getSynchronizedState();
+ try {
+ getStateMachine().doTransition(event.getType(), event);
+ stageState = getSynchronizedState();
+ } catch (InvalidStateTransitonException e) {
+ LOG.error("Can't handle this event at current state"
+ + ", eventType:" + event.getType().name()
+ + ", oldState:" + oldState.name()
+ + ", nextState:" + getSynchronizedState().name()
+ , e);
+ eventHandler.handle(new StageEvent(getId(),
+ StageEventType.SQ_INTERNAL_ERROR));
+ }
+
+ // notify the eventhandler of state change
+ if (LOG.isDebugEnabled()) {
+ if (oldState != getSynchronizedState()) {
+ LOG.debug(getId() + " Stage Transitioned from " + oldState + " to "
+ + getSynchronizedState());
+ }
+ }
+ } finally {
+ writeLock.unlock();
+ }
+ }
+
+ public void handleTaskRequestEvent(TaskRequestEvent event) {
+ taskScheduler.handleTaskRequestEvent(event);
+ }
+
+ private static class InitAndRequestContainer implements MultipleArcTransition<Stage,
+ StageEvent, StageState> {
+
+ @Override
+ public StageState transition(final Stage stage, StageEvent stageEvent) {
+ stage.setStartTime();
+ ExecutionBlock execBlock = stage.getBlock();
+ StageState state;
+
+ try {
+ // Union operator does not require actual query processing. It is performed logically.
+ if (execBlock.hasUnion()) {
+ stage.finalizeStats();
+ state = StageState.SUCCEEDED;
+ } else {
+ ExecutionBlock parent = stage.getMasterPlan().getParent(stage.getBlock());
+ DataChannel channel = stage.getMasterPlan().getChannel(stage.getId(), parent.getId());
+ setShuffleIfNecessary(stage, channel);
+ initTaskScheduler(stage);
+ // execute pre-processing asyncronously
+ stage.getContext().getQueryMasterContext().getEventExecutor()
+ .submit(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ schedule(stage);
+ stage.totalScheduledObjectsCount = stage.getTaskScheduler().remainingScheduledObjectNum();
+ LOG.info(stage.totalScheduledObjectsCount + " objects are scheduled");
+
+ if (stage.getTaskScheduler().remainingScheduledObjectNum() == 0) { // if there is no tasks
+ stage.complete();
+ } else {
+ if(stage.getSynchronizedState() == StageState.INITED) {
+ stage.taskScheduler.start();
+ allocateContainers(stage);
+ } else {
+ stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_KILL));
+ }
+ }
+ } catch (Throwable e) {
+ LOG.error("Stage (" + stage.getId() + ") ERROR: ", e);
+ stage.setFinishTime();
+ stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(), e.getMessage()));
+ stage.eventHandler.handle(new StageCompletedEvent(stage.getId(), StageState.ERROR));
+ }
+ }
+ }
+ );
+ state = StageState.INITED;
+ }
+ } catch (Throwable e) {
+ LOG.error("Stage (" + stage.getId() + ") ERROR: ", e);
+ stage.setFinishTime();
+ stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(), e.getMessage()));
+ stage.eventHandler.handle(new StageCompletedEvent(stage.getId(), StageState.ERROR));
+ return StageState.ERROR;
+ }
+
+ return state;
+ }
+
+ private void initTaskScheduler(Stage stage) throws IOException {
+ TajoConf conf = stage.context.getConf();
+ stage.schedulerContext = new TaskSchedulerContext(stage.context,
+ stage.getMasterPlan().isLeaf(stage.getId()), stage.getId());
+ stage.taskScheduler = TaskSchedulerFactory.get(conf, stage.schedulerContext, stage);
+ stage.taskScheduler.init(conf);
+ LOG.info(stage.taskScheduler.getName() + " is chosen for the task scheduling for " + stage.getId());
+ }
+
+ /**
+ * If a parent block requires a repartition operation, the method sets proper repartition
+ * methods and the number of partitions to a given Stage.
+ */
+ private static void setShuffleIfNecessary(Stage stage, DataChannel channel) {
+ if (channel.getShuffleType() != ShuffleType.NONE_SHUFFLE) {
+ int numTasks = calculateShuffleOutputNum(stage, channel);
+ Repartitioner.setShuffleOutputNumForTwoPhase(stage, numTasks, channel);
+ }
+ }
+
+ /**
+ * Getting the total memory of cluster
+ *
+ * @param stage
+ * @return mega bytes
+ */
+ private static int getClusterTotalMemory(Stage stage) {
+ List<TajoMasterProtocol.WorkerResourceProto> workers =
+ stage.context.getQueryMasterContext().getQueryMaster().getAllWorker();
+
+ int totalMem = 0;
+ for (TajoMasterProtocol.WorkerResourceProto worker : workers) {
+ totalMem += worker.getMemoryMB();
+ }
+ return totalMem;
+ }
+ /**
+ * Getting the desire number of partitions according to the volume of input data.
+ * This method is only used to determine the partition key number of hash join or aggregation.
+ *
+ * @param stage
+ * @return
+ */
+ public static int calculateShuffleOutputNum(Stage stage, DataChannel channel) {
+ TajoConf conf = stage.context.getConf();
+ MasterPlan masterPlan = stage.getMasterPlan();
+ ExecutionBlock parent = masterPlan.getParent(stage.getBlock());
+
+ LogicalNode grpNode = null;
+ if (parent != null) {
+ grpNode = PlannerUtil.findMostBottomNode(parent.getPlan(), NodeType.GROUP_BY);
+ if (grpNode == null) {
+ grpNode = PlannerUtil.findMostBottomNode(parent.getPlan(), NodeType.DISTINCT_GROUP_BY);
+ }
+ }
+
+ // We assume this execution block the first stage of join if two or more tables are included in this block,
+ if (parent != null && parent.getScanNodes().length >= 2) {
+ List<ExecutionBlock> childs = masterPlan.getChilds(parent);
+
+ // for outer
+ ExecutionBlock outer = childs.get(0);
+ long outerVolume = getInputVolume(stage.masterPlan, stage.context, outer);
+
+ // for inner
+ ExecutionBlock inner = childs.get(1);
+ long innerVolume = getInputVolume(stage.masterPlan, stage.context, inner);
+ LOG.info(stage.getId() + ", Outer volume: " + Math.ceil((double) outerVolume / 1048576) + "MB, "
+ + "Inner volume: " + Math.ceil((double) innerVolume / 1048576) + "MB");
+
+ long bigger = Math.max(outerVolume, innerVolume);
+
+ int mb = (int) Math.ceil((double) bigger / 1048576);
+ LOG.info(stage.getId() + ", Bigger Table's volume is approximately " + mb + " MB");
+
+ int taskNum = (int) Math.ceil((double) mb / masterPlan.getContext().getInt(SessionVars.JOIN_PER_SHUFFLE_SIZE));
+
+ if (masterPlan.getContext().containsKey(SessionVars.TEST_MIN_TASK_NUM)) {
+ taskNum = masterPlan.getContext().getInt(SessionVars.TEST_MIN_TASK_NUM);
+ LOG.warn("!!!!! TESTCASE MODE !!!!!");
+ }
+
+ // The shuffle output numbers of join may be inconsistent by execution block order.
+ // Thus, we need to compare the number with DataChannel output numbers.
+ // If the number is right, the number and DataChannel output numbers will be consistent.
+ int outerShuffleOutputNum = 0, innerShuffleOutputNum = 0;
+ for (DataChannel eachChannel : masterPlan.getOutgoingChannels(outer.getId())) {
+ outerShuffleOutputNum = Math.max(outerShuffleOutputNum, eachChannel.getShuffleOutputNum());
+ }
+ for (DataChannel eachChannel : masterPlan.getOutgoingChannels(inner.getId())) {
+ innerShuffleOutputNum = Math.max(innerShuffleOutputNum, eachChannel.getShuffleOutputNum());
+ }
+ if (outerShuffleOutputNum != innerShuffleOutputNum
+ && taskNum != outerShuffleOutputNum
+ && taskNum != innerShuffleOutputNum) {
+ LOG.info(stage.getId() + ", Change determined number of join partitions cause difference of outputNum" +
+ ", originTaskNum=" + taskNum + ", changedTaskNum=" + Math.max(outerShuffleOutputNum, innerShuffleOutputNum) +
+ ", outerShuffleOutptNum=" + outerShuffleOutputNum +
+ ", innerShuffleOutputNum=" + innerShuffleOutputNum);
+ taskNum = Math.max(outerShuffleOutputNum, innerShuffleOutputNum);
+ }
+
+ LOG.info(stage.getId() + ", The determined number of join partitions is " + taskNum);
+
+ return taskNum;
+ // Is this stage the first step of group-by?
+ } else if (grpNode != null) {
+ boolean hasGroupColumns = true;
+ if (grpNode.getType() == NodeType.GROUP_BY) {
+ hasGroupColumns = ((GroupbyNode)grpNode).getGroupingColumns().length > 0;
+ } else if (grpNode.getType() == NodeType.DISTINCT_GROUP_BY) {
+ // Find current distinct stage node.
+ DistinctGroupbyNode distinctNode = PlannerUtil.findMostBottomNode(stage.getBlock().getPlan(), NodeType.DISTINCT_GROUP_BY);
+ if (distinctNode == null) {
+ LOG.warn(stage.getId() + ", Can't find current DistinctGroupbyNode");
+ distinctNode = (DistinctGroupbyNode)grpNode;
+ }
+ hasGroupColumns = distinctNode.getGroupingColumns().length > 0;
+
+ Enforcer enforcer = stage.getBlock().getEnforcer();
+ if (enforcer == null) {
+ LOG.warn(stage.getId() + ", DistinctGroupbyNode's enforcer is null.");
+ }
+ EnforceProperty property = PhysicalPlannerImpl.getAlgorithmEnforceProperty(enforcer, distinctNode);
+ if (property != null) {
+ if (property.getDistinct().getIsMultipleAggregation()) {
+ MultipleAggregationStage multiAggStage = property.getDistinct().getMultipleAggregationStage();
+ if (multiAggStage != MultipleAggregationStage.THRID_STAGE) {
+ hasGroupColumns = true;
+ }
+ }
+ }
+ }
+ if (!hasGroupColumns) {
+ LOG.info(stage.getId() + ", No Grouping Column - determinedTaskNum is set to 1");
+ return 1;
+ } else {
+ long volume = getInputVolume(stage.masterPlan, stage.context, stage.block);
+
+ int volumeByMB = (int) Math.ceil((double) volume / StorageUnit.MB);
+ LOG.info(stage.getId() + ", Table's volume is approximately " + volumeByMB + " MB");
+ // determine the number of task
+ int taskNum = (int) Math.ceil((double) volumeByMB /
+ masterPlan.getContext().getInt(SessionVars.GROUPBY_PER_SHUFFLE_SIZE));
+ LOG.info(stage.getId() + ", The determined number of aggregation partitions is " + taskNum);
+ return taskNum;
+ }
+ } else {
+ LOG.info("============>>>>> Unexpected Case! <<<<<================");
+ long volume = getInputVolume(stage.masterPlan, stage.context, stage.block);
+
+ int mb = (int) Math.ceil((double)volume / 1048576);
+ LOG.info(stage.getId() + ", Table's volume is approximately " + mb + " MB");
+ // determine the number of task per 128MB
+ int taskNum = (int) Math.ceil((double)mb / 128);
+ LOG.info(stage.getId() + ", The determined number of partitions is " + taskNum);
+ return taskNum;
+ }
+ }
+
+ private static void schedule(Stage stage) throws IOException {
+ MasterPlan masterPlan = stage.getMasterPlan();
+ ExecutionBlock execBlock = stage.getBlock();
+ if (stage.getMasterPlan().isLeaf(execBlock.getId()) && execBlock.getScanNodes().length == 1) { // Case 1: Just Scan
+ scheduleFragmentsForLeafQuery(stage);
+ } else if (execBlock.getScanNodes().length > 1) { // Case 2: Join
+ Repartitioner.scheduleFragmentsForJoinQuery(stage.schedulerContext, stage);
+ } else { // Case 3: Others (Sort or Aggregation)
+ int numTasks = getNonLeafTaskNum(stage);
+ Repartitioner.scheduleFragmentsForNonLeafTasks(stage.schedulerContext, masterPlan, stage, numTasks);
+ }
+ }
+
+ /**
+ * Getting the desire number of tasks according to the volume of input data
+ *
+ * @param stage
+ * @return
+ */
+ public static int getNonLeafTaskNum(Stage stage) {
+ // Getting intermediate data size
+ long volume = getInputVolume(stage.getMasterPlan(), stage.context, stage.getBlock());
+
+ int mb = (int) Math.ceil((double)volume / 1048576);
+ LOG.info(stage.getId() + ", Table's volume is approximately " + mb + " MB");
+ // determine the number of task per 64MB
+ int maxTaskNum = Math.max(1, (int) Math.ceil((double)mb / 64));
+ LOG.info(stage.getId() + ", The determined number of non-leaf tasks is " + maxTaskNum);
+ return maxTaskNum;
+ }
+
+ public static long getInputVolume(MasterPlan masterPlan, QueryMasterTask.QueryMasterTaskContext context,
+ ExecutionBlock execBlock) {
+ Map<String, TableDesc> tableMap = context.getTableDescMap();
+ if (masterPlan.isLeaf(execBlock)) {
+ ScanNode[] outerScans = execBlock.getScanNodes();
+ long maxVolume = 0;
+ for (ScanNode eachScanNode: outerScans) {
+ TableStats stat = tableMap.get(eachScanNode.getCanonicalName()).getStats();
+ if (stat.getNumBytes() > maxVolume) {
+ maxVolume = stat.getNumBytes();
+ }
+ }
+ return maxVolume;
+ } else {
+ long aggregatedVolume = 0;
+ for (ExecutionBlock childBlock : masterPlan.getChilds(execBlock)) {
+ Stage stage = context.getStage(childBlock.getId());
+ if (stage == null || stage.getSynchronizedState() != StageState.SUCCEEDED) {
+ aggregatedVolume += getInputVolume(masterPlan, context, childBlock);
+ } else {
+ aggregatedVolume += stage.getResultStats().getNumBytes();
+ }
+ }
+
+ return aggregatedVolume;
+ }
+ }
+
+ public static void allocateContainers(Stage stage) {
+ ExecutionBlock execBlock = stage.getBlock();
+
+ //TODO consider disk slot
+ int requiredMemoryMBPerTask = 512;
+
+ int numRequest = stage.getContext().getResourceAllocator().calculateNumRequestContainers(
+ stage.getContext().getQueryMasterContext().getWorkerContext(),
+ stage.schedulerContext.getEstimatedTaskNum(),
+ requiredMemoryMBPerTask
+ );
+
+ final Resource resource = Records.newRecord(Resource.class);
+
+ resource.setMemory(requiredMemoryMBPerTask);
+
+ LOG.info("Request Container for " + stage.getId() + " containers=" + numRequest);
+
+ Priority priority = Records.newRecord(Priority.class);
+ priority.setPriority(stage.getPriority());
+ ContainerAllocationEvent event =
+ new ContainerAllocationEvent(ContainerAllocatorEventType.CONTAINER_REQ,
+ stage.getId(), priority, resource, numRequest,
+ stage.masterPlan.isLeaf(execBlock), 0.0f);
+ stage.eventHandler.handle(event);
+ }
+
+ private static void scheduleFragmentsForLeafQuery(Stage stage) throws IOException {
+ ExecutionBlock execBlock = stage.getBlock();
+ ScanNode[] scans = execBlock.getScanNodes();
+ Preconditions.checkArgument(scans.length == 1, "Must be Scan Query");
+ ScanNode scan = scans[0];
+ TableDesc table = stage.context.getTableDescMap().get(scan.getCanonicalName());
+
+ Collection<Fragment> fragments;
+ TableMeta meta = table.getMeta();
+
+ // Depending on scanner node's type, it creates fragments. If scan is for
+ // a partitioned table, It will creates lots fragments for all partitions.
+ // Otherwise, it creates at least one fragments for a table, which may
+ // span a number of blocks or possibly consists of a number of files.
+ if (scan.getType() == NodeType.PARTITIONS_SCAN) {
+ // After calling this method, partition paths are removed from the physical plan.
+ FileStorageManager storageManager =
+ (FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf());
+ fragments = Repartitioner.getFragmentsFromPartitionedTable(storageManager, scan, table);
+ } else {
+ StorageManager storageManager =
+ StorageManager.getStorageManager(stage.getContext().getConf(), meta.getStoreType());
+ fragments = storageManager.getSplits(scan.getCanonicalName(), table, scan);
+ }
+
+ Stage.scheduleFragments(stage, fragments);
+ if (stage.getTaskScheduler() instanceof DefaultTaskScheduler) {
+ //Leaf task of DefaultTaskScheduler should be fragment size
+ // EstimatedTaskNum determined number of initial container
+ stage.schedulerContext.setEstimatedTaskNum(fragments.size());
+ } else {
+ TajoConf conf = stage.context.getConf();
+ stage.schedulerContext.setTaskSize(conf.getIntVar(ConfVars.TASK_DEFAULT_SIZE) * 1024 * 1024);
+ int estimatedTaskNum = (int) Math.ceil((double) table.getStats().getNumBytes() /
+ (double) stage.schedulerContext.getTaskSize());
+ stage.schedulerContext.setEstimatedTaskNum(estimatedTaskNum);
+ }
+ }
+ }
+
+ public static void scheduleFragment(Stage stage, Fragment fragment) {
+ stage.taskScheduler.handle(new FragmentScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
+ stage.getId(), fragment));
+ }
+
+
+ public static void scheduleFragments(Stage stage, Collection<Fragment> fragments) {
+ for (Fragment eachFragment : fragments) {
+ scheduleFragment(stage, eachFragment);
+ }
+ }
+
+ public static void scheduleFragments(Stage stage, Collection<Fragment> leftFragments,
+ Collection<Fragment> broadcastFragments) {
+ for (Fragment eachLeafFragment : leftFragments) {
+ scheduleFragment(stage, eachLeafFragment, broadcastFragments);
+ }
+ }
+
+ public static void scheduleFragment(Stage stage,
+ Fragment leftFragment, Collection<Fragment> rightFragments) {
+ stage.taskScheduler.handle(new FragmentScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
+ stage.getId(), leftFragment, rightFragments));
+ }
+
+ public static void scheduleFetches(Stage stage, Map<String, List<FetchImpl>> fetches) {
+ stage.taskScheduler.handle(new FetchScheduleEvent(TaskSchedulerEvent.EventType.T_SCHEDULE,
+ stage.getId(), fetches));
+ }
+
+ public static Task newEmptyTask(TaskSchedulerContext schedulerContext,
+ TaskAttemptScheduleContext taskContext,
+ Stage stage, int taskId) {
+ ExecutionBlock execBlock = stage.getBlock();
+ Task unit = new Task(schedulerContext.getMasterContext().getConf(),
+ taskContext,
+ QueryIdFactory.newTaskId(schedulerContext.getBlockId(), taskId),
+ schedulerContext.isLeafQuery(), stage.eventHandler);
+ unit.setLogicalPlan(execBlock.getPlan());
+ stage.addTask(unit);
+ return unit;
+ }
+
+ private static class ContainerLaunchTransition
+ implements SingleArcTransition<Stage, StageEvent> {
+
+ @Override
+ public void transition(Stage stage, StageEvent event) {
+ try {
+ StageContainerAllocationEvent allocationEvent =
+ (StageContainerAllocationEvent) event;
+ for (TajoContainer container : allocationEvent.getAllocatedContainer()) {
+ TajoContainerId cId = container.getId();
+ if (stage.containers.containsKey(cId)) {
+ stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(),
+ "Duplicated containers are allocated: " + cId.toString()));
+ stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_INTERNAL_ERROR));
+ }
+ stage.containers.put(cId, container);
+ }
+ LOG.info("Stage (" + stage.getId() + ") has " + stage.containers.size() + " containers!");
+ stage.eventHandler.handle(
+ new LaunchTaskRunnersEvent(stage.getId(), allocationEvent.getAllocatedContainer(),
+ stage.getContext().getQueryContext(),
+ CoreGsonHelper.toJson(stage.getBlock().getPlan(), LogicalNode.class))
+ );
+
+ stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_START));
+ } catch (Throwable t) {
+ stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(),
+ ExceptionUtils.getStackTrace(t)));
+ stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_INTERNAL_ERROR));
+ }
+ }
+ }
+
+ /**
+ * It is used in KILL_WAIT state against Contained Allocated event.
+ * It just returns allocated containers to resource manager.
+ */
+ private static class AllocatedContainersCancelTransition implements SingleArcTransition<Stage, StageEvent> {
+ @Override
+ public void transition(Stage stage, StageEvent event) {
+ try {
+ StageContainerAllocationEvent allocationEvent =
+ (StageContainerAllocationEvent) event;
+ stage.eventHandler.handle(
+ new TaskRunnerGroupEvent(EventType.CONTAINER_REMOTE_CLEANUP,
+ stage.getId(), allocationEvent.getAllocatedContainer()));
+ LOG.info(String.format("[%s] %d allocated containers are canceled",
+ stage.getId().toString(),
+ allocationEvent.getAllocatedContainer().size()));
+ } catch (Throwable t) {
+ stage.eventHandler.handle(new StageDiagnosticsUpdateEvent(stage.getId(),
+ ExceptionUtils.getStackTrace(t)));
+ stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_INTERNAL_ERROR));
+ }
+ }
+ }
+
+ private static class TaskCompletedTransition implements SingleArcTransition<Stage, StageEvent> {
+
+ @Override
+ public void transition(Stage stage,
+ StageEvent event) {
+ StageTaskEvent taskEvent = (StageTaskEvent) event;
+ Task task = stage.getTask(taskEvent.getTaskId());
+
+ if (task == null) { // task failed
+ LOG.error(String.format("Task %s is absent", taskEvent.getTaskId()));
+ stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_FAILED));
+ } else {
+ stage.completedTaskCount++;
+
+ if (taskEvent.getState() == TaskState.SUCCEEDED) {
+ stage.succeededObjectCount++;
+ } else if (task.getState() == TaskState.KILLED) {
+ stage.killedObjectCount++;
+ } else if (task.getState() == TaskState.FAILED) {
+ stage.failedObjectCount++;
+ // if at least one task is failed, try to kill all tasks.
+ stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_KILL));
+ }
+
+ LOG.info(String.format("[%s] Task Completion Event (Total: %d, Success: %d, Killed: %d, Failed: %d)",
+ stage.getId(),
+ stage.getTotalScheduledObjectsCount(),
+ stage.succeededObjectCount,
+ stage.killedObjectCount,
+ stage.failedObjectCount));
+
+ if (stage.totalScheduledObjectsCount ==
+ stage.succeededObjectCount + stage.killedObjectCount + stage.failedObjectCount) {
+ stage.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_STAGE_COMPLETED));
+ }
+ }
+ }
+ }
+
+ private static class KillTasksTransition implements SingleArcTransition<Stage, StageEvent> {
+
+ @Override
+ public void transition(Stage stage, StageEvent stageEvent) {
+ if(stage.getTaskScheduler() != null){
+ stage.getTaskScheduler().stop();
+ }
+
+ for (Task task : stage.getTasks()) {
+ stage.eventHandler.handle(new TaskEvent(task.getId(), TaskEventType.T_KILL));
+ }
+ }
+ }
+
+ private void cleanup() {
+ stopScheduler();
+ releaseContainers();
+
+ if (!getContext().getQueryContext().getBool(SessionVars.DEBUG_ENABLED)) {
+ List<ExecutionBlock> childs = getMasterPlan().getChilds(getId());
+ List<TajoIdProtos.ExecutionBlockIdProto> ebIds = Lists.newArrayList();
+
+ for (ExecutionBlock executionBlock : childs) {
+ ebIds.add(executionBlock.getId().getProto());
+ }
+
+ getContext().getQueryMasterContext().getQueryMaster().cleanupExecutionBlock(ebIds);
+ }
+
+ this.finalStageHistory = makeStageHistory();
+ this.finalStageHistory.setTasks(makeTaskHistories());
+ }
+
+ public List<IntermediateEntry> getHashShuffleIntermediateEntries() {
+ return hashShuffleIntermediateEntries;
+ }
+
+ protected void waitingIntermediateReport() {
+ LOG.info(getId() + ", waiting IntermediateReport: expectedTaskNum=" + completeReportReceived.get());
+ synchronized(completeReportReceived) {
+ long startTime = System.currentTimeMillis();
+ while (true) {
+ if (completeReportReceived.get() >= tasks.size()) {
+ LOG.info(getId() + ", completed waiting IntermediateReport");
+ return;
+ } else {
+ try {
+ completeReportReceived.wait(10 * 1000);
+ } catch (InterruptedException e) {
+ }
+ long elapsedTime = System.currentTimeMillis() - startTime;
+ if (elapsedTime >= 120 * 1000) {
+ LOG.error(getId() + ", Timeout while receiving intermediate reports: " + elapsedTime + " ms");
+ abort(StageState.FAILED);
+ return;
+ }
+ }
+ }
+ }
+ }
+
+ public void receiveExecutionBlockReport(TajoWorkerProtocol.ExecutionBlockReport report) {
+ LOG.info(getId() + ", receiveExecutionBlockReport:" + report.getSucceededTasks());
+ if (!report.getReportSuccess()) {
+ LOG.error(getId() + ", ExecutionBlock final report fail cause:" + report.getReportErrorMessage());
+ abort(StageState.FAILED);
+ return;
+ }
+ if (report.getIntermediateEntriesCount() > 0) {
+ synchronized (hashShuffleIntermediateEntries) {
+ for (IntermediateEntryProto eachInterm: report.getIntermediateEntriesList()) {
+ hashShuffleIntermediateEntries.add(new IntermediateEntry(eachInterm));
+ }
+ }
+ }
+ synchronized(completeReportReceived) {
+ completeReportReceived.addAndGet(report.getSucceededTasks());
+ completeReportReceived.notifyAll();
+ }
+ }
+
+ private static class StageCompleteTransition implements MultipleArcTransition<Stage, StageEvent, StageState> {
+
+ @Override
+ public StageState transition(Stage stage, StageEvent stageEvent) {
+ // TODO - Commit Stage
+ // TODO - records succeeded, failed, killed completed task
+ // TODO - records metrics
+ try {
+ LOG.info(String.format("Stage completed - %s (total=%d, success=%d, killed=%d)",
+ stage.getId().toString(),
+ stage.getTotalScheduledObjectsCount(),
+ stage.getSucceededObjectCount(),
+ stage.killedObjectCount));
+
+ if (stage.killedObjectCount > 0 || stage.failedObjectCount > 0) {
+ if (stage.failedObjectCount > 0) {
+ stage.abort(StageState.FAILED);
+ return StageState.FAILED;
+ } else if (stage.killedObjectCount > 0) {
+ stage.abort(StageState.KILLED);
+ return StageState.KILLED;
+ } else {
+ LOG.error("Invalid State " + stage.getSynchronizedState() + " State");
+ stage.abort(StageState.ERROR);
+ return StageState.ERROR;
+ }
+ } else {
+ stage.complete();
+ return StageState.SUCCEEDED;
+ }
+ } catch (Throwable t) {
+ LOG.error(t.getMessage(), t);
+ stage.abort(StageState.ERROR);
+ return StageState.ERROR;
+ }
+ }
+ }
+
+ private static class DiagnosticsUpdateTransition implements SingleArcTransition<Stage, StageEvent> {
+ @Override
+ public void transition(Stage stage, StageEvent event) {
+ stage.addDiagnostic(((StageDiagnosticsUpdateEvent) event).getDiagnosticUpdate());
+ }
+ }
+
+ private static class InternalErrorTransition implements SingleArcTransition<Stage, StageEvent> {
+ @Override
+ public void transition(Stage stage, StageEvent stageEvent) {
+ stage.abort(StageState.ERROR);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/StageState.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/StageState.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/StageState.java
new file mode 100644
index 0000000..2fd62be
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/StageState.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.tajo.querymaster;
+
+public enum StageState {
+ NEW,
+ INITED,
+ RUNNING,
+ SUCCEEDED,
+ FAILED,
+ KILL_WAIT,
+ KILLED,
+ ERROR
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java
new file mode 100644
index 0000000..1c6a9a3
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Task.java
@@ -0,0 +1,897 @@
+/**
+ * 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.tajo.querymaster;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.state.*;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.TajoProtos.TaskAttemptState;
+import org.apache.tajo.TaskAttemptId;
+import org.apache.tajo.TaskId;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.ipc.TajoWorkerProtocol.FailureIntermediateProto;
+import org.apache.tajo.ipc.TajoWorkerProtocol.IntermediateEntryProto;
+import org.apache.tajo.master.TaskState;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext;
+import org.apache.tajo.plan.logical.*;
+import org.apache.tajo.storage.DataLocation;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.fragment.FragmentConvertor;
+import org.apache.tajo.util.Pair;
+import org.apache.tajo.util.TajoIdUtils;
+import org.apache.tajo.util.history.TaskHistory;
+import org.apache.tajo.worker.FetchImpl;
+
+import java.net.URI;
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleFileOutput;
+
+public class Task implements EventHandler<TaskEvent> {
+ /** Class Logger */
+ private static final Log LOG = LogFactory.getLog(Task.class);
+
+ private final Configuration systemConf;
+ private TaskId taskId;
+ private EventHandler eventHandler;
+ private StoreTableNode store = null;
+ private LogicalNode plan = null;
+ private List<ScanNode> scan;
+
+ private Map<String, Set<FragmentProto>> fragMap;
+ private Map<String, Set<FetchImpl>> fetchMap;
+
+ private int totalFragmentNum;
+
+ private List<ShuffleFileOutput> shuffleFileOutputs;
+ private TableStats stats;
+ private final boolean isLeafTask;
+ private List<IntermediateEntry> intermediateData;
+
+ private Map<TaskAttemptId, TaskAttempt> attempts;
+ private final int maxAttempts = 3;
+ private Integer nextAttempt = -1;
+ private TaskAttemptId lastAttemptId;
+
+ private TaskAttemptId successfulAttempt;
+ private String succeededHost;
+ private int succeededHostPort;
+ private int succeededPullServerPort;
+
+ private int failedAttempts;
+ private int finishedAttempts; // finish are total of success, failed and killed
+
+ private long launchTime;
+ private long finishTime;
+
+ private List<DataLocation> dataLocations = Lists.newArrayList();
+
+ private static final AttemptKilledTransition ATTEMPT_KILLED_TRANSITION = new AttemptKilledTransition();
+
+ private TaskHistory finalTaskHistory;
+
+ protected static final StateMachineFactory
+ <Task, TaskState, TaskEventType, TaskEvent> stateMachineFactory =
+ new StateMachineFactory <Task, TaskState, TaskEventType, TaskEvent>(TaskState.NEW)
+
+ // Transitions from NEW state
+ .addTransition(TaskState.NEW, TaskState.SCHEDULED,
+ TaskEventType.T_SCHEDULE,
+ new InitialScheduleTransition())
+ .addTransition(TaskState.NEW, TaskState.KILLED,
+ TaskEventType.T_KILL,
+ new KillNewTaskTransition())
+
+ // Transitions from SCHEDULED state
+ .addTransition(TaskState.SCHEDULED, TaskState.RUNNING,
+ TaskEventType.T_ATTEMPT_LAUNCHED,
+ new AttemptLaunchedTransition())
+ .addTransition(TaskState.SCHEDULED, TaskState.KILL_WAIT,
+ TaskEventType.T_KILL,
+ new KillTaskTransition())
+
+ // Transitions from RUNNING state
+ .addTransition(TaskState.RUNNING, TaskState.RUNNING,
+ TaskEventType.T_ATTEMPT_LAUNCHED)
+ .addTransition(TaskState.RUNNING, TaskState.SUCCEEDED,
+ TaskEventType.T_ATTEMPT_SUCCEEDED,
+ new AttemptSucceededTransition())
+ .addTransition(TaskState.RUNNING, TaskState.KILL_WAIT,
+ TaskEventType.T_KILL,
+ new KillTaskTransition())
+ .addTransition(TaskState.RUNNING,
+ EnumSet.of(TaskState.RUNNING, TaskState.FAILED),
+ TaskEventType.T_ATTEMPT_FAILED,
+ new AttemptFailedOrRetryTransition())
+
+ // Transitions from KILL_WAIT state
+ .addTransition(TaskState.KILL_WAIT, TaskState.KILLED,
+ TaskEventType.T_ATTEMPT_KILLED,
+ ATTEMPT_KILLED_TRANSITION)
+ .addTransition(TaskState.KILL_WAIT, TaskState.KILL_WAIT,
+ TaskEventType.T_ATTEMPT_LAUNCHED,
+ new KillTaskTransition())
+ .addTransition(TaskState.KILL_WAIT, TaskState.FAILED,
+ TaskEventType.T_ATTEMPT_FAILED,
+ new AttemptFailedTransition())
+ .addTransition(TaskState.KILL_WAIT, TaskState.KILLED,
+ TaskEventType.T_ATTEMPT_SUCCEEDED,
+ ATTEMPT_KILLED_TRANSITION)
+ // Ignore-able transitions.
+ .addTransition(TaskState.KILL_WAIT, TaskState.KILL_WAIT,
+ EnumSet.of(
+ TaskEventType.T_KILL,
+ TaskEventType.T_SCHEDULE))
+
+ // Transitions from SUCCEEDED state
+ // Ignore-able transitions
+ .addTransition(TaskState.SUCCEEDED, TaskState.SUCCEEDED,
+ EnumSet.of(TaskEventType.T_KILL,
+ TaskEventType.T_ATTEMPT_KILLED, TaskEventType.T_ATTEMPT_SUCCEEDED, TaskEventType.T_ATTEMPT_FAILED))
+
+ // Transitions from FAILED state
+ // Ignore-able transitions
+ .addTransition(TaskState.FAILED, TaskState.FAILED,
+ EnumSet.of(TaskEventType.T_KILL,
+ TaskEventType.T_ATTEMPT_KILLED, TaskEventType.T_ATTEMPT_SUCCEEDED, TaskEventType.T_ATTEMPT_FAILED))
+
+ // Transitions from KILLED state
+ .addTransition(TaskState.KILLED, TaskState.KILLED, TaskEventType.T_ATTEMPT_KILLED, new KillTaskTransition())
+ // Ignore-able transitions
+ .addTransition(TaskState.KILLED, TaskState.KILLED,
+ EnumSet.of(
+ TaskEventType.T_KILL, TaskEventType.T_ATTEMPT_SUCCEEDED, TaskEventType.T_ATTEMPT_FAILED))
+
+ .installTopology();
+
+ private final StateMachine<TaskState, TaskEventType, TaskEvent> stateMachine;
+
+
+ private final Lock readLock;
+ private final Lock writeLock;
+ private TaskAttemptScheduleContext scheduleContext;
+
+ public Task(Configuration conf, TaskAttemptScheduleContext scheduleContext,
+ TaskId id, boolean isLeafTask, EventHandler eventHandler) {
+ this.systemConf = conf;
+ this.taskId = id;
+ this.eventHandler = eventHandler;
+ this.isLeafTask = isLeafTask;
+ scan = new ArrayList<ScanNode>();
+ fetchMap = Maps.newHashMap();
+ fragMap = Maps.newHashMap();
+ shuffleFileOutputs = new ArrayList<ShuffleFileOutput>();
+ attempts = Collections.emptyMap();
+ lastAttemptId = null;
+ nextAttempt = -1;
+ failedAttempts = 0;
+
+ ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+ this.readLock = readWriteLock.readLock();
+ this.writeLock = readWriteLock.writeLock();
+ this.scheduleContext = scheduleContext;
+
+ stateMachine = stateMachineFactory.make(this);
+ totalFragmentNum = 0;
+ }
+
+ public boolean isLeafTask() {
+ return this.isLeafTask;
+ }
+
+ public TaskState getState() {
+ readLock.lock();
+ try {
+ return stateMachine.getCurrentState();
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ public TaskAttemptState getLastAttemptStatus() {
+ TaskAttempt lastAttempt = getLastAttempt();
+ if (lastAttempt != null) {
+ return lastAttempt.getState();
+ } else {
+ return TaskAttemptState.TA_ASSIGNED;
+ }
+ }
+
+ public TaskHistory getTaskHistory() {
+ if (finalTaskHistory != null) {
+ if (finalTaskHistory.getFinishTime() == 0) {
+ finalTaskHistory = makeTaskHistory();
+ }
+ return finalTaskHistory;
+ } else {
+ return makeTaskHistory();
+ }
+ }
+
+ private TaskHistory makeTaskHistory() {
+ TaskHistory taskHistory = new TaskHistory();
+
+ TaskAttempt lastAttempt = getLastAttempt();
+ if (lastAttempt != null) {
+ taskHistory.setId(lastAttempt.getId().toString());
+ taskHistory.setState(lastAttempt.getState().toString());
+ taskHistory.setProgress(lastAttempt.getProgress());
+ }
+ taskHistory.setHostAndPort(succeededHost + ":" + succeededHostPort);
+ taskHistory.setRetryCount(this.getRetryCount());
+ taskHistory.setLaunchTime(launchTime);
+ taskHistory.setFinishTime(finishTime);
+
+ taskHistory.setNumShuffles(getShuffleOutpuNum());
+ if (!getShuffleFileOutputs().isEmpty()) {
+ ShuffleFileOutput shuffleFileOutputs = getShuffleFileOutputs().get(0);
+ if (taskHistory.getNumShuffles() > 0) {
+ taskHistory.setShuffleKey("" + shuffleFileOutputs.getPartId());
+ taskHistory.setShuffleFileName(shuffleFileOutputs.getFileName());
+ }
+ }
+
+ List<String> fragmentList = new ArrayList<String>();
+ for (FragmentProto eachFragment : getAllFragments()) {
+ try {
+ Fragment fragment = FragmentConvertor.convert(systemConf, eachFragment);
+ fragmentList.add(fragment.toString());
+ } catch (Exception e) {
+ LOG.error(e.getMessage());
+ fragmentList.add("ERROR: " + eachFragment.getStoreType() + "," + eachFragment.getId() + ": " + e.getMessage());
+ }
+ }
+ taskHistory.setFragments(fragmentList.toArray(new String[]{}));
+
+ List<String[]> fetchList = new ArrayList<String[]>();
+ for (Map.Entry<String, Set<FetchImpl>> e : getFetchMap().entrySet()) {
+ for (FetchImpl f : e.getValue()) {
+ for (URI uri : f.getSimpleURIs()){
+ fetchList.add(new String[] {e.getKey(), uri.toString()});
+ }
+ }
+ }
+
+ taskHistory.setFetchs(fetchList.toArray(new String[][]{}));
+
+ List<String> dataLocationList = new ArrayList<String>();
+ for(DataLocation eachLocation: getDataLocations()) {
+ dataLocationList.add(eachLocation.toString());
+ }
+
+ taskHistory.setDataLocations(dataLocationList.toArray(new String[]{}));
+ return taskHistory;
+ }
+
+ public void setLogicalPlan(LogicalNode plan) {
+ this.plan = plan;
+
+ LogicalNode node = plan;
+ ArrayList<LogicalNode> s = new ArrayList<LogicalNode>();
+ s.add(node);
+ while (!s.isEmpty()) {
+ node = s.remove(s.size()-1);
+ if (node instanceof UnaryNode) {
+ UnaryNode unary = (UnaryNode) node;
+ s.add(s.size(), unary.getChild());
+ } else if (node instanceof BinaryNode) {
+ BinaryNode binary = (BinaryNode) node;
+ s.add(s.size(), binary.getLeftChild());
+ s.add(s.size(), binary.getRightChild());
+ } else if (node instanceof ScanNode) {
+ scan.add((ScanNode)node);
+ } else if (node instanceof TableSubQueryNode) {
+ s.add(((TableSubQueryNode) node).getSubQuery());
+ }
+ }
+ }
+
+ private void addDataLocation(Fragment fragment) {
+ String[] hosts = fragment.getHosts();
+ int[] diskIds = null;
+ if (fragment instanceof FileFragment) {
+ diskIds = ((FileFragment)fragment).getDiskIds();
+ }
+ for (int i = 0; i < hosts.length; i++) {
+ dataLocations.add(new DataLocation(hosts[i], diskIds == null ? -1 : diskIds[i]));
+ }
+ }
+
+ public void addFragment(Fragment fragment, boolean useDataLocation) {
+ Set<FragmentProto> fragmentProtos;
+ if (fragMap.containsKey(fragment.getTableName())) {
+ fragmentProtos = fragMap.get(fragment.getTableName());
+ } else {
+ fragmentProtos = new HashSet<FragmentProto>();
+ fragMap.put(fragment.getTableName(), fragmentProtos);
+ }
+ fragmentProtos.add(fragment.getProto());
+ if (useDataLocation) {
+ addDataLocation(fragment);
+ }
+ totalFragmentNum++;
+ }
+
+ public void addFragments(Collection<Fragment> fragments) {
+ for (Fragment eachFragment: fragments) {
+ addFragment(eachFragment, false);
+ }
+ }
+
+ public List<DataLocation> getDataLocations() {
+ return dataLocations;
+ }
+
+ public String getSucceededHost() {
+ return succeededHost;
+ }
+
+ public void addFetches(String tableId, Collection<FetchImpl> fetches) {
+ Set<FetchImpl> fetchSet;
+ if (fetchMap.containsKey(tableId)) {
+ fetchSet = fetchMap.get(tableId);
+ } else {
+ fetchSet = Sets.newHashSet();
+ }
+ fetchSet.addAll(fetches);
+ fetchMap.put(tableId, fetchSet);
+ }
+
+ public void setFetches(Map<String, Set<FetchImpl>> fetches) {
+ this.fetchMap.clear();
+ this.fetchMap.putAll(fetches);
+ }
+
+ public Collection<FragmentProto> getAllFragments() {
+ Set<FragmentProto> fragmentProtos = new HashSet<FragmentProto>();
+ for (Set<FragmentProto> eachFragmentSet : fragMap.values()) {
+ fragmentProtos.addAll(eachFragmentSet);
+ }
+ return fragmentProtos;
+ }
+
+ public LogicalNode getLogicalPlan() {
+ return this.plan;
+ }
+
+ public TaskId getId() {
+ return taskId;
+ }
+
+ public Collection<FetchImpl> getFetchHosts(String tableId) {
+ return fetchMap.get(tableId);
+ }
+
+ public Collection<Set<FetchImpl>> getFetches() {
+ return fetchMap.values();
+ }
+
+ public Map<String, Set<FetchImpl>> getFetchMap() {
+ return fetchMap;
+ }
+
+ public Collection<FetchImpl> getFetch(ScanNode scan) {
+ return this.fetchMap.get(scan.getTableName());
+ }
+
+ public ScanNode[] getScanNodes() {
+ return this.scan.toArray(new ScanNode[scan.size()]);
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ builder.append(plan.getType() + " \n");
+ for (Entry<String, Set<FragmentProto>> e : fragMap.entrySet()) {
+ builder.append(e.getKey()).append(" : ");
+ for (FragmentProto fragment : e.getValue()) {
+ builder.append(fragment).append(", ");
+ }
+ }
+ for (Entry<String, Set<FetchImpl>> e : fetchMap.entrySet()) {
+ builder.append(e.getKey()).append(" : ");
+ for (FetchImpl t : e.getValue()) {
+ for (URI uri : t.getURIs()){
+ builder.append(uri).append(" ");
+ }
+ }
+ }
+
+ return builder.toString();
+ }
+
+ public void setStats(TableStats stats) {
+ this.stats = stats;
+ }
+
+ public void setShuffleFileOutputs(List<ShuffleFileOutput> partitions) {
+ this.shuffleFileOutputs = Collections.unmodifiableList(partitions);
+ }
+
+ public TableStats getStats() {
+ return this.stats;
+ }
+
+ public List<ShuffleFileOutput> getShuffleFileOutputs() {
+ return this.shuffleFileOutputs;
+ }
+
+ public int getShuffleOutpuNum() {
+ return this.shuffleFileOutputs.size();
+ }
+
+ public TaskAttempt newAttempt() {
+ TaskAttempt attempt = new TaskAttempt(scheduleContext,
+ QueryIdFactory.newTaskAttemptId(this.getId(), ++nextAttempt),
+ this, eventHandler);
+ lastAttemptId = attempt.getId();
+ return attempt;
+ }
+
+ public TaskAttempt getAttempt(TaskAttemptId attemptId) {
+ return attempts.get(attemptId);
+ }
+
+ public TaskAttempt getAttempt(int attempt) {
+ return this.attempts.get(QueryIdFactory.newTaskAttemptId(this.getId(), attempt));
+ }
+
+ public TaskAttempt getLastAttempt() {
+ return getAttempt(this.lastAttemptId);
+ }
+
+ public TaskAttempt getSuccessfulAttempt() {
+ readLock.lock();
+ try {
+ if (null == successfulAttempt) {
+ return null;
+ }
+ return attempts.get(successfulAttempt);
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ public int getRetryCount () {
+ return this.nextAttempt;
+ }
+
+ public int getTotalFragmentNum() {
+ return totalFragmentNum;
+ }
+
+ private static class InitialScheduleTransition implements
+ SingleArcTransition<Task, TaskEvent> {
+
+ @Override
+ public void transition(Task task, TaskEvent taskEvent) {
+ task.addAndScheduleAttempt();
+ }
+ }
+
+ public long getLaunchTime() {
+ return launchTime;
+ }
+
+ public long getFinishTime() {
+ return finishTime;
+ }
+
+ @VisibleForTesting
+ public void setLaunchTime(long launchTime) {
+ this.launchTime = launchTime;
+ }
+
+ @VisibleForTesting
+ public void setFinishTime(long finishTime) {
+ this.finishTime = finishTime;
+ }
+
+ public long getRunningTime() {
+ if(finishTime > 0) {
+ return finishTime - launchTime;
+ } else {
+ return System.currentTimeMillis() - launchTime;
+ }
+ }
+
+ // This is always called in the Write Lock
+ private void addAndScheduleAttempt() {
+ // Create new task attempt
+ TaskAttempt attempt = newAttempt();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Created attempt " + attempt.getId());
+ }
+ switch (attempts.size()) {
+ case 0:
+ attempts = Collections.singletonMap(attempt.getId(), attempt);
+ break;
+
+ case 1:
+ Map<TaskAttemptId, TaskAttempt> newAttempts
+ = new LinkedHashMap<TaskAttemptId, TaskAttempt>(3);
+ newAttempts.putAll(attempts);
+ attempts = newAttempts;
+ attempts.put(attempt.getId(), attempt);
+ break;
+
+ default:
+ attempts.put(attempt.getId(), attempt);
+ break;
+ }
+
+ if (failedAttempts > 0) {
+ eventHandler.handle(new TaskAttemptScheduleEvent(systemConf, attempt.getId(),
+ TaskAttemptEventType.TA_RESCHEDULE));
+ } else {
+ eventHandler.handle(new TaskAttemptScheduleEvent(systemConf, attempt.getId(),
+ TaskAttemptEventType.TA_SCHEDULE));
+ }
+ }
+
+ private void finishTask() {
+ this.finishTime = System.currentTimeMillis();
+ finalTaskHistory = makeTaskHistory();
+ }
+
+ private static class KillNewTaskTransition implements SingleArcTransition<Task, TaskEvent> {
+
+ @Override
+ public void transition(Task task, TaskEvent taskEvent) {
+ task.eventHandler.handle(new StageTaskEvent(task.getId(), TaskState.KILLED));
+ }
+ }
+
+ private static class KillTaskTransition implements SingleArcTransition<Task, TaskEvent> {
+
+ @Override
+ public void transition(Task task, TaskEvent taskEvent) {
+ task.finishTask();
+ task.eventHandler.handle(new TaskAttemptEvent(task.lastAttemptId, TaskAttemptEventType.TA_KILL));
+ }
+ }
+
+ private static class AttemptKilledTransition implements SingleArcTransition<Task, TaskEvent>{
+
+ @Override
+ public void transition(Task task, TaskEvent event) {
+ task.finishTask();
+ task.eventHandler.handle(new StageTaskEvent(task.getId(), TaskState.KILLED));
+ }
+ }
+
+ private static class AttemptSucceededTransition
+ implements SingleArcTransition<Task, TaskEvent>{
+
+ @Override
+ public void transition(Task task,
+ TaskEvent event) {
+ TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) event;
+ TaskAttempt attempt = task.attempts.get(attemptEvent.getTaskAttemptId());
+
+ task.successfulAttempt = attemptEvent.getTaskAttemptId();
+ task.succeededHost = attempt.getWorkerConnectionInfo().getHost();
+ task.succeededHostPort = attempt.getWorkerConnectionInfo().getPeerRpcPort();
+ task.succeededPullServerPort = attempt.getWorkerConnectionInfo().getPullServerPort();
+
+ task.finishTask();
+ task.eventHandler.handle(new StageTaskEvent(event.getTaskId(), TaskState.SUCCEEDED));
+ }
+ }
+
+ private static class AttemptLaunchedTransition implements SingleArcTransition<Task, TaskEvent> {
+ @Override
+ public void transition(Task task,
+ TaskEvent event) {
+ TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) event;
+ TaskAttempt attempt = task.attempts.get(attemptEvent.getTaskAttemptId());
+ task.launchTime = System.currentTimeMillis();
+ task.succeededHost = attempt.getWorkerConnectionInfo().getHost();
+ task.succeededHostPort = attempt.getWorkerConnectionInfo().getPeerRpcPort();
+ }
+ }
+
+ private static class AttemptFailedTransition implements SingleArcTransition<Task, TaskEvent> {
+ @Override
+ public void transition(Task task, TaskEvent event) {
+ TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) event;
+ LOG.info("=============================================================");
+ LOG.info(">>> Task Failed: " + attemptEvent.getTaskAttemptId() + " <<<");
+ LOG.info("=============================================================");
+ task.failedAttempts++;
+ task.finishedAttempts++;
+
+ task.finishTask();
+ task.eventHandler.handle(new StageTaskEvent(task.getId(), TaskState.FAILED));
+ }
+ }
+
+ private static class AttemptFailedOrRetryTransition implements
+ MultipleArcTransition<Task, TaskEvent, TaskState> {
+
+ @Override
+ public TaskState transition(Task task, TaskEvent taskEvent) {
+ TaskTAttemptEvent attemptEvent = (TaskTAttemptEvent) taskEvent;
+ task.failedAttempts++;
+ task.finishedAttempts++;
+ boolean retry = task.failedAttempts < task.maxAttempts;
+
+ LOG.info("====================================================================================");
+ LOG.info(">>> Task Failed: " + attemptEvent.getTaskAttemptId() + ", " +
+ "retry:" + retry + ", attempts:" + task.failedAttempts + " <<<");
+ LOG.info("====================================================================================");
+
+ if (retry) {
+ if (task.successfulAttempt == null) {
+ task.addAndScheduleAttempt();
+ }
+ } else {
+ task.finishTask();
+ task.eventHandler.handle(new StageTaskEvent(task.getId(), TaskState.FAILED));
+ return TaskState.FAILED;
+ }
+
+ return task.getState();
+ }
+ }
+
+ @Override
+ public void handle(TaskEvent event) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Processing " + event.getTaskId() + " of type "
+ + event.getType());
+ }
+
+ try {
+ writeLock.lock();
+ TaskState oldState = getState();
+ try {
+ stateMachine.doTransition(event.getType(), event);
+ } catch (InvalidStateTransitonException e) {
+ LOG.error("Can't handle this event at current state"
+ + ", eventType:" + event.getType().name()
+ + ", oldState:" + oldState.name()
+ + ", nextState:" + getState().name()
+ , e);
+ eventHandler.handle(new QueryEvent(TajoIdUtils.parseQueryId(getId().toString()),
+ QueryEventType.INTERNAL_ERROR));
+ }
+
+ //notify the eventhandler of state change
+ if (LOG.isDebugEnabled()) {
+ if (oldState != getState()) {
+ LOG.debug(taskId + " Task Transitioned from " + oldState + " to "
+ + getState());
+ }
+ }
+ }
+
+ finally {
+ writeLock.unlock();
+ }
+ }
+
+ public void setIntermediateData(Collection<IntermediateEntry> partitions) {
+ this.intermediateData = new ArrayList<IntermediateEntry>(partitions);
+ }
+
+ public List<IntermediateEntry> getIntermediateData() {
+ return this.intermediateData;
+ }
+
+ public static class PullHost implements Cloneable {
+ String host;
+ int port;
+ int hashCode;
+
+ public PullHost(String pullServerAddr, int pullServerPort){
+ this.host = pullServerAddr;
+ this.port = pullServerPort;
+ this.hashCode = Objects.hashCode(host, port);
+ }
+ public String getHost() {
+ return host;
+ }
+
+ public int getPort() {
+ return this.port;
+ }
+
+ public String getPullAddress() {
+ return host + ":" + port;
+ }
+
+ @Override
+ public int hashCode() {
+ return hashCode;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof PullHost) {
+ PullHost other = (PullHost) obj;
+ return host.equals(other.host) && port == other.port;
+ }
+
+ return false;
+ }
+
+ @Override
+ public PullHost clone() throws CloneNotSupportedException {
+ PullHost newPullHost = (PullHost) super.clone();
+ newPullHost.host = host;
+ newPullHost.port = port;
+ newPullHost.hashCode = Objects.hashCode(newPullHost.host, newPullHost.port);
+ return newPullHost;
+ }
+
+ @Override
+ public String toString() {
+ return host + ":" + port;
+ }
+ }
+
+ public static class IntermediateEntry {
+ ExecutionBlockId ebId;
+ int taskId;
+ int attemptId;
+ int partId;
+ PullHost host;
+ long volume;
+ List<Pair<Long, Integer>> pages;
+ List<Pair<Long, Pair<Integer, Integer>>> failureRowNums;
+
+ public IntermediateEntry(IntermediateEntryProto proto) {
+ this.ebId = new ExecutionBlockId(proto.getEbId());
+ this.taskId = proto.getTaskId();
+ this.attemptId = proto.getAttemptId();
+ this.partId = proto.getPartId();
+
+ String[] pullHost = proto.getHost().split(":");
+ this.host = new PullHost(pullHost[0], Integer.parseInt(pullHost[1]));
+ this.volume = proto.getVolume();
+
+ failureRowNums = new ArrayList<Pair<Long, Pair<Integer, Integer>>>();
+ for (FailureIntermediateProto eachFailure: proto.getFailuresList()) {
+
+ failureRowNums.add(new Pair(eachFailure.getPagePos(),
+ new Pair(eachFailure.getStartRowNum(), eachFailure.getEndRowNum())));
+ }
+
+ pages = new ArrayList<Pair<Long, Integer>>();
+ for (IntermediateEntryProto.PageProto eachPage: proto.getPagesList()) {
+ pages.add(new Pair(eachPage.getPos(), eachPage.getLength()));
+ }
+ }
+
+ public IntermediateEntry(int taskId, int attemptId, int partId, PullHost host) {
+ this.taskId = taskId;
+ this.attemptId = attemptId;
+ this.partId = partId;
+ this.host = host;
+ }
+
+ public IntermediateEntry(int taskId, int attemptId, int partId, PullHost host, long volume) {
+ this.taskId = taskId;
+ this.attemptId = attemptId;
+ this.partId = partId;
+ this.host = host;
+ this.volume = volume;
+ }
+
+ public ExecutionBlockId getEbId() {
+ return ebId;
+ }
+
+ public void setEbId(ExecutionBlockId ebId) {
+ this.ebId = ebId;
+ }
+
+ public int getTaskId() {
+ return this.taskId;
+ }
+
+ public int getAttemptId() {
+ return this.attemptId;
+ }
+
+ public int getPartId() {
+ return this.partId;
+ }
+
+ public PullHost getPullHost() {
+ return this.host;
+ }
+
+ public long getVolume() {
+ return this.volume;
+ }
+
+ public long setVolume(long volume) {
+ return this.volume = volume;
+ }
+
+ public List<Pair<Long, Integer>> getPages() {
+ return pages;
+ }
+
+ public void setPages(List<Pair<Long, Integer>> pages) {
+ this.pages = pages;
+ }
+
+ public List<Pair<Long, Pair<Integer, Integer>>> getFailureRowNums() {
+ return failureRowNums;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hashCode(ebId, taskId, partId, attemptId, host);
+ }
+
+ public List<Pair<Long, Long>> split(long firstSplitVolume, long splitVolume) {
+ List<Pair<Long, Long>> splits = new ArrayList<Pair<Long, Long>>();
+
+ if (pages == null || pages.isEmpty()) {
+ return splits;
+ }
+ int pageSize = pages.size();
+
+ long currentOffset = -1;
+ long currentBytes = 0;
+
+ long realSplitVolume = firstSplitVolume > 0 ? firstSplitVolume : splitVolume;
+ for (int i = 0; i < pageSize; i++) {
+ Pair<Long, Integer> eachPage = pages.get(i);
+ if (currentOffset == -1) {
+ currentOffset = eachPage.getFirst();
+ }
+ if (currentBytes > 0 && currentBytes + eachPage.getSecond() >= realSplitVolume) {
+ splits.add(new Pair(currentOffset, currentBytes));
+ currentOffset = eachPage.getFirst();
+ currentBytes = 0;
+ realSplitVolume = splitVolume;
+ }
+
+ currentBytes += eachPage.getSecond();
+ }
+
+ //add last
+ if (currentBytes > 0) {
+ splits.add(new Pair(currentOffset, currentBytes));
+ }
+ return splits;
+ }
+ }
+}
[08/16] tajo git commit: TAJO-1288: Refactoring
org.apache.tajo.master package.
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/TaskAttempt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/TaskAttempt.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/TaskAttempt.java
deleted file mode 100644
index 0f161ff..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/TaskAttempt.java
+++ /dev/null
@@ -1,443 +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.tajo.master.querymaster;
-
-import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.state.*;
-import org.apache.tajo.TaskAttemptId;
-import org.apache.tajo.TajoProtos.TaskAttemptState;
-import org.apache.tajo.catalog.proto.CatalogProtos;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.ipc.TajoWorkerProtocol.TaskCompletionReport;
-import org.apache.tajo.master.cluster.WorkerConnectionInfo;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext;
-import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
-import org.apache.tajo.master.querymaster.Task.IntermediateEntry;
-import org.apache.tajo.master.querymaster.Task.PullHost;
-import org.apache.tajo.master.container.TajoContainerId;
-
-import java.util.ArrayList;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleFileOutput;
-
-public class TaskAttempt implements EventHandler<TaskAttemptEvent> {
-
- private static final Log LOG = LogFactory.getLog(TaskAttempt.class);
-
- private final static int EXPIRE_TIME = 15000;
-
- private final TaskAttemptId id;
- private final Task task;
- final EventHandler eventHandler;
-
- private TajoContainerId containerId;
- private WorkerConnectionInfo workerConnectionInfo;
- private int expire;
-
- private final Lock readLock;
- private final Lock writeLock;
-
- private final List<String> diagnostics = new ArrayList<String>();
-
- private final TaskAttemptScheduleContext scheduleContext;
-
- private float progress;
- private CatalogProtos.TableStatsProto inputStats;
- private CatalogProtos.TableStatsProto resultStats;
-
- protected static final StateMachineFactory
- <TaskAttempt, TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
- stateMachineFactory = new StateMachineFactory
- <TaskAttempt, TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
- (TaskAttemptState.TA_NEW)
-
- // Transitions from TA_NEW state
- .addTransition(TaskAttemptState.TA_NEW, TaskAttemptState.TA_UNASSIGNED,
- TaskAttemptEventType.TA_SCHEDULE, new TaskAttemptScheduleTransition())
- .addTransition(TaskAttemptState.TA_NEW, TaskAttemptState.TA_UNASSIGNED,
- TaskAttemptEventType.TA_RESCHEDULE, new TaskAttemptScheduleTransition())
- .addTransition(TaskAttemptState.TA_NEW, TaskAttemptState.TA_KILLED,
- TaskAttemptEventType.TA_KILL,
- new TaskKilledCompleteTransition())
-
- // Transitions from TA_UNASSIGNED state
- .addTransition(TaskAttemptState.TA_UNASSIGNED, TaskAttemptState.TA_ASSIGNED,
- TaskAttemptEventType.TA_ASSIGNED,
- new LaunchTransition())
- .addTransition(TaskAttemptState.TA_UNASSIGNED, TaskAttemptState.TA_KILL_WAIT,
- TaskAttemptEventType.TA_KILL,
- new KillUnassignedTaskTransition())
-
- // Transitions from TA_ASSIGNED state
- .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_ASSIGNED,
- TaskAttemptEventType.TA_ASSIGNED, new AlreadyAssignedTransition())
- .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_KILL_WAIT,
- TaskAttemptEventType.TA_KILL,
- new KillTaskTransition())
- .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_KILLED,
- TaskAttemptEventType.TA_KILL,
- new KillTaskTransition())
- .addTransition(TaskAttemptState.TA_ASSIGNED,
- EnumSet.of(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_KILLED),
- TaskAttemptEventType.TA_UPDATE, new StatusUpdateTransition())
- .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_SUCCEEDED,
- TaskAttemptEventType.TA_DONE, new SucceededTransition())
- .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_FAILED,
- TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
-
- // Transitions from TA_RUNNING state
- .addTransition(TaskAttemptState.TA_RUNNING,
- EnumSet.of(TaskAttemptState.TA_RUNNING),
- TaskAttemptEventType.TA_UPDATE, new StatusUpdateTransition())
- .addTransition(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_KILL_WAIT,
- TaskAttemptEventType.TA_KILL,
- new KillTaskTransition())
- .addTransition(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_SUCCEEDED,
- TaskAttemptEventType.TA_DONE, new SucceededTransition())
- .addTransition(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_FAILED,
- TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
-
- .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILLED,
- TaskAttemptEventType.TA_LOCAL_KILLED,
- new TaskKilledCompleteTransition())
- .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILL_WAIT,
- TaskAttemptEventType.TA_ASSIGNED,
- new KillTaskTransition())
- .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILLED,
- TaskAttemptEventType.TA_SCHEDULE_CANCELED,
- new TaskKilledCompleteTransition())
- .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILLED,
- TaskAttemptEventType.TA_DONE,
- new TaskKilledCompleteTransition())
- .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_FAILED,
- TaskAttemptEventType.TA_FATAL_ERROR)
- .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILL_WAIT,
- EnumSet.of(
- TaskAttemptEventType.TA_KILL,
- TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
- TaskAttemptEventType.TA_UPDATE))
-
- // Transitions from TA_SUCCEEDED state
- .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_SUCCEEDED,
- TaskAttemptEventType.TA_UPDATE)
- .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_SUCCEEDED,
- TaskAttemptEventType.TA_DONE, new AlreadyDoneTransition())
- .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_FAILED,
- TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
- // Ignore-able transitions
- .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_SUCCEEDED,
- TaskAttemptEventType.TA_KILL)
-
- // Transitions from TA_KILLED state
- .addTransition(TaskAttemptState.TA_KILLED, TaskAttemptState.TA_KILLED,
- TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE)
- // Ignore-able transitions
- .addTransition(TaskAttemptState.TA_KILLED, TaskAttemptState.TA_KILLED,
- EnumSet.of(
- TaskAttemptEventType.TA_UPDATE))
- .addTransition(TaskAttemptState.TA_KILLED, TaskAttemptState.TA_KILLED,
- EnumSet.of(
- TaskAttemptEventType.TA_LOCAL_KILLED,
- TaskAttemptEventType.TA_KILL,
- TaskAttemptEventType.TA_ASSIGNED,
- TaskAttemptEventType.TA_DONE),
- new TaskKilledCompleteTransition())
- .installTopology();
-
- private final StateMachine<TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
- stateMachine;
-
-
- public TaskAttempt(final TaskAttemptScheduleContext scheduleContext,
- final TaskAttemptId id, final Task task,
- final EventHandler eventHandler) {
- this.scheduleContext = scheduleContext;
- this.id = id;
- this.expire = TaskAttempt.EXPIRE_TIME;
- this.task = task;
- this.eventHandler = eventHandler;
-
- ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
- this.readLock = readWriteLock.readLock();
- this.writeLock = readWriteLock.writeLock();
-
- stateMachine = stateMachineFactory.make(this);
- }
-
- public TaskAttemptState getState() {
- readLock.lock();
- try {
- return stateMachine.getCurrentState();
- } finally {
- readLock.unlock();
- }
- }
-
- public TaskAttemptId getId() {
- return this.id;
- }
-
- public boolean isLeafTask() {
- return this.task.isLeafTask();
- }
-
- public Task getTask() {
- return this.task;
- }
-
- public WorkerConnectionInfo getWorkerConnectionInfo() {
- return this.workerConnectionInfo;
- }
-
- public void setContainerId(TajoContainerId containerId) {
- this.containerId = containerId;
- }
-
- public synchronized void setExpireTime(int expire) {
- this.expire = expire;
- }
-
- public synchronized void updateExpireTime(int period) {
- this.setExpireTime(this.expire - period);
- }
-
- public synchronized void resetExpireTime() {
- this.setExpireTime(TaskAttempt.EXPIRE_TIME);
- }
-
- public int getLeftTime() {
- return this.expire;
- }
-
- public float getProgress() {
- return progress;
- }
-
- public TableStats getInputStats() {
- if (inputStats == null) {
- return null;
- }
-
- return new TableStats(inputStats);
- }
-
- public TableStats getResultStats() {
- if (resultStats == null) {
- return null;
- }
- return new TableStats(resultStats);
- }
-
- private void fillTaskStatistics(TaskCompletionReport report) {
- this.progress = 1.0f;
-
- List<IntermediateEntry> partitions = new ArrayList<IntermediateEntry>();
-
- if (report.getShuffleFileOutputsCount() > 0) {
- this.getTask().setShuffleFileOutputs(report.getShuffleFileOutputsList());
-
- PullHost host = new PullHost(getWorkerConnectionInfo().getHost(), getWorkerConnectionInfo().getPullServerPort());
- for (ShuffleFileOutput p : report.getShuffleFileOutputsList()) {
- IntermediateEntry entry = new IntermediateEntry(getId().getTaskId().getId(),
- getId().getId(), p.getPartId(), host, p.getVolume());
- partitions.add(entry);
- }
- }
- this.getTask().setIntermediateData(partitions);
-
- if (report.hasInputStats()) {
- this.inputStats = report.getInputStats();
- }
- if (report.hasResultStats()) {
- this.resultStats = report.getResultStats();
- this.getTask().setStats(new TableStats(resultStats));
- }
- }
-
- private static class TaskAttemptScheduleTransition implements
- SingleArcTransition<TaskAttempt, TaskAttemptEvent> {
-
- @Override
- public void transition(TaskAttempt taskAttempt, TaskAttemptEvent taskAttemptEvent) {
- taskAttempt.eventHandler.handle(new TaskAttemptToSchedulerEvent(
- EventType.T_SCHEDULE, taskAttempt.getTask().getId().getExecutionBlockId(),
- taskAttempt.scheduleContext, taskAttempt));
- }
- }
-
- private static class KillUnassignedTaskTransition implements
- SingleArcTransition<TaskAttempt, TaskAttemptEvent> {
-
- @Override
- public void transition(TaskAttempt taskAttempt, TaskAttemptEvent taskAttemptEvent) {
- taskAttempt.eventHandler.handle(new TaskAttemptToSchedulerEvent(
- EventType.T_SCHEDULE_CANCEL, taskAttempt.getTask().getId().getExecutionBlockId(),
- taskAttempt.scheduleContext, taskAttempt));
- }
- }
-
- private static class LaunchTransition
- implements SingleArcTransition<TaskAttempt, TaskAttemptEvent> {
-
- @Override
- public void transition(TaskAttempt taskAttempt,
- TaskAttemptEvent event) {
- TaskAttemptAssignedEvent castEvent = (TaskAttemptAssignedEvent) event;
- taskAttempt.containerId = castEvent.getContainerId();
- taskAttempt.workerConnectionInfo = castEvent.getWorkerConnectionInfo();
- taskAttempt.eventHandler.handle(
- new TaskTAttemptEvent(taskAttempt.getId(),
- TaskEventType.T_ATTEMPT_LAUNCHED));
- }
- }
-
- private static class TaskKilledCompleteTransition implements SingleArcTransition<TaskAttempt, TaskAttemptEvent> {
-
- @Override
- public void transition(TaskAttempt taskAttempt,
- TaskAttemptEvent event) {
- taskAttempt.getTask().handle(new TaskEvent(taskAttempt.getId().getTaskId(),
- TaskEventType.T_ATTEMPT_KILLED));
- LOG.info(taskAttempt.getId() + " Received TA_KILLED Status from LocalTask");
- }
- }
-
- private static class StatusUpdateTransition
- implements MultipleArcTransition<TaskAttempt, TaskAttemptEvent, TaskAttemptState> {
-
- @Override
- public TaskAttemptState transition(TaskAttempt taskAttempt,
- TaskAttemptEvent event) {
- TaskAttemptStatusUpdateEvent updateEvent = (TaskAttemptStatusUpdateEvent) event;
-
- taskAttempt.progress = updateEvent.getStatus().getProgress();
- taskAttempt.inputStats = updateEvent.getStatus().getInputStats();
- taskAttempt.resultStats = updateEvent.getStatus().getResultStats();
-
- return TaskAttemptState.TA_RUNNING;
- }
- }
-
- private void addDiagnosticInfo(String diag) {
- if (diag != null && !diag.equals("")) {
- diagnostics.add(diag);
- }
- }
-
- private static class AlreadyAssignedTransition
- implements SingleArcTransition<TaskAttempt, TaskAttemptEvent>{
-
- @Override
- public void transition(TaskAttempt taskAttempt,
- TaskAttemptEvent taskAttemptEvent) {
- }
- }
-
- private static class AlreadyDoneTransition
- implements SingleArcTransition<TaskAttempt, TaskAttemptEvent>{
-
- @Override
- public void transition(TaskAttempt taskAttempt,
- TaskAttemptEvent taskAttemptEvent) {
- }
- }
-
- private static class SucceededTransition implements SingleArcTransition<TaskAttempt, TaskAttemptEvent> {
- @Override
- public void transition(TaskAttempt taskAttempt,
- TaskAttemptEvent event) {
- TaskCompletionReport report = ((TaskCompletionEvent)event).getReport();
-
- try {
- taskAttempt.fillTaskStatistics(report);
- taskAttempt.eventHandler.handle(new TaskTAttemptEvent(taskAttempt.getId(), TaskEventType.T_ATTEMPT_SUCCEEDED));
- } catch (Throwable t) {
- taskAttempt.eventHandler.handle(new TaskFatalErrorEvent(taskAttempt.getId(), t.getMessage()));
- taskAttempt.addDiagnosticInfo(ExceptionUtils.getStackTrace(t));
- }
- }
- }
-
- private static class KillTaskTransition implements SingleArcTransition<TaskAttempt, TaskAttemptEvent> {
-
- @Override
- public void transition(TaskAttempt taskAttempt, TaskAttemptEvent event) {
- taskAttempt.eventHandler.handle(new LocalTaskEvent(taskAttempt.getId(), taskAttempt.containerId,
- LocalTaskEventType.KILL));
- }
- }
-
- private static class FailedTransition implements SingleArcTransition<TaskAttempt, TaskAttemptEvent>{
- @Override
- public void transition(TaskAttempt taskAttempt, TaskAttemptEvent event) {
- TaskFatalErrorEvent errorEvent = (TaskFatalErrorEvent) event;
- taskAttempt.eventHandler.handle(new TaskTAttemptEvent(taskAttempt.getId(), TaskEventType.T_ATTEMPT_FAILED));
- taskAttempt.addDiagnosticInfo(errorEvent.errorMessage());
- LOG.error(taskAttempt.getId() + " FROM " + taskAttempt.getWorkerConnectionInfo().getHost()
- + " >> " + errorEvent.errorMessage());
- }
- }
-
- @Override
- public void handle(TaskAttemptEvent event) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Processing " + event.getTaskAttemptId() + " of type " + event.getType());
- }
- try {
- writeLock.lock();
- TaskAttemptState oldState = getState();
- try {
- stateMachine.doTransition(event.getType(), event);
- } catch (InvalidStateTransitonException e) {
- LOG.error("Can't handle this event at current state of " + event.getTaskAttemptId() + ")"
- + ", eventType:" + event.getType().name()
- + ", oldState:" + oldState.name()
- + ", nextState:" + getState().name()
- , e);
- eventHandler.handle(
- new StageDiagnosticsUpdateEvent(event.getTaskAttemptId().getTaskId().getExecutionBlockId(),
- "Can't handle this event at current state of " + event.getTaskAttemptId() + ")"));
- eventHandler.handle(
- new StageEvent(event.getTaskAttemptId().getTaskId().getExecutionBlockId(),
- StageEventType.SQ_INTERNAL_ERROR));
- }
-
- //notify the eventhandler of state change
- if (LOG.isDebugEnabled()) {
- if (oldState != getState()) {
- LOG.debug(id + " TaskAttempt Transitioned from " + oldState + " to "
- + getState());
- }
- }
- }
-
- finally {
- writeLock.unlock();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
index f1a9224..c4200d5 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java
@@ -36,10 +36,9 @@ import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.ipc.ContainerProtocol;
import org.apache.tajo.ipc.TajoMasterProtocol;
import org.apache.tajo.master.TajoMaster;
-import org.apache.tajo.master.querymaster.QueryInProgress;
+import org.apache.tajo.querymaster.QueryInProgress;
import org.apache.tajo.rpc.CallFuture;
import org.apache.tajo.util.ApplicationIdUtils;
-import org.apache.tajo.util.StringUtils;
import java.io.IOException;
import java.util.*;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
index 9c2b71b..b237cc5 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/WorkerResourceManager.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.service.Service;
import org.apache.tajo.QueryId;
import org.apache.tajo.ipc.ContainerProtocol;
import org.apache.tajo.ipc.TajoMasterProtocol;
-import org.apache.tajo.master.querymaster.QueryInProgress;
+import org.apache.tajo.querymaster.QueryInProgress;
import java.io.IOException;
import java.util.Collection;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java
new file mode 100644
index 0000000..3dd3389
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/QuerySchedulingInfo.java
@@ -0,0 +1,55 @@
+/**
+ * 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.tajo.master.scheduler;
+
+import com.google.common.base.Objects;
+import org.apache.tajo.QueryId;
+
+public class QuerySchedulingInfo {
+ private QueryId queryId;
+ private Integer priority;
+ private Long startTime;
+
+ public QuerySchedulingInfo(QueryId queryId, Integer priority, Long startTime) {
+ this.queryId = queryId;
+ this.priority = priority;
+ this.startTime = startTime;
+ }
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public Integer getPriority() {
+ return priority;
+ }
+
+ public Long getStartTime() {
+ return startTime;
+ }
+
+ public String getName() {
+ return queryId.getId();
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hashCode(startTime, getName(), priority);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/scheduler/Scheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/Scheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/Scheduler.java
new file mode 100644
index 0000000..02203a9
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/Scheduler.java
@@ -0,0 +1,41 @@
+/**
+ * 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.tajo.master.scheduler;
+
+import org.apache.tajo.QueryId;
+import org.apache.tajo.querymaster.QueryInProgress;
+
+import java.util.List;
+
+public interface Scheduler {
+
+ public Mode getMode();
+
+ public String getName();
+
+ public boolean addQuery(QueryInProgress resource);
+
+ public boolean removeQuery(QueryId queryId);
+
+ public List<QueryInProgress> getRunningQueries();
+
+ public enum Mode {
+ FIFO
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SchedulingAlgorithms.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SchedulingAlgorithms.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SchedulingAlgorithms.java
new file mode 100644
index 0000000..7fd07b5
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SchedulingAlgorithms.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.tajo.master.scheduler;
+
+import java.util.Comparator;
+
+/**
+ * Utility class containing scheduling algorithms used in the scheduler.
+ */
+
+public class SchedulingAlgorithms {
+ /**
+ * Compare Schedulables in order of priority and then submission time, as in
+ * the default FIFO scheduler in Tajo.
+ */
+ public static class FifoComparator implements Comparator<QuerySchedulingInfo> {
+ @Override
+ public int compare(QuerySchedulingInfo q1, QuerySchedulingInfo q2) {
+ int res = q1.getPriority().compareTo(q2.getPriority());
+ if (res == 0) {
+ res = (int) Math.signum(q1.getStartTime() - q2.getStartTime());
+ }
+ if (res == 0) {
+ // In the rare case where jobs were submitted at the exact same time,
+ // compare them by name (which will be the QueryId) to get a deterministic ordering
+ res = q1.getName().compareTo(q2.getName());
+ }
+ return res;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleFifoScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleFifoScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleFifoScheduler.java
new file mode 100644
index 0000000..bd8ca28
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/scheduler/SimpleFifoScheduler.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.tajo.master.scheduler;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.querymaster.QueryInProgress;
+import org.apache.tajo.master.QueryJobManager;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class SimpleFifoScheduler implements Scheduler {
+ private static final Log LOG = LogFactory.getLog(SimpleFifoScheduler.class.getName());
+ private LinkedList<QuerySchedulingInfo> pool = new LinkedList<QuerySchedulingInfo>();
+ private final Thread queryProcessor;
+ private AtomicBoolean stopped = new AtomicBoolean();
+ private QueryJobManager manager;
+ private Comparator<QuerySchedulingInfo> COMPARATOR = new SchedulingAlgorithms.FifoComparator();
+
+ public SimpleFifoScheduler(QueryJobManager manager) {
+ this.manager = manager;
+ this.queryProcessor = new Thread(new QueryProcessor());
+ this.queryProcessor.setName("Query Processor");
+ }
+
+ @Override
+ public Mode getMode() {
+ return Mode.FIFO;
+ }
+
+ @Override
+ public String getName() {
+ return manager.getName();
+ }
+
+ @Override
+ public boolean addQuery(QueryInProgress queryInProgress) {
+ int qSize = pool.size();
+ if (qSize != 0 && qSize % 100 == 0) {
+ LOG.info("Size of Fifo queue is " + qSize);
+ }
+
+ QuerySchedulingInfo querySchedulingInfo = new QuerySchedulingInfo(queryInProgress.getQueryId(), 1, queryInProgress.getStartTime());
+ boolean result = pool.add(querySchedulingInfo);
+ if (getRunningQueries().size() == 0) wakeupProcessor();
+ return result;
+ }
+
+ @Override
+ public boolean removeQuery(QueryId queryId) {
+ return pool.remove(getQueryByQueryId(queryId));
+ }
+
+ public QuerySchedulingInfo getQueryByQueryId(QueryId queryId) {
+ for (QuerySchedulingInfo querySchedulingInfo : pool) {
+ if (querySchedulingInfo.getQueryId().equals(queryId)) {
+ return querySchedulingInfo;
+ }
+ }
+ return null;
+ }
+
+ @Override
+ public List<QueryInProgress> getRunningQueries() {
+ return new ArrayList<QueryInProgress>(manager.getRunningQueries());
+ }
+
+ public void start() {
+ queryProcessor.start();
+ }
+
+ public void stop() {
+ if (stopped.getAndSet(true)) {
+ return;
+ }
+ pool.clear();
+ synchronized (queryProcessor) {
+ queryProcessor.interrupt();
+ }
+ }
+
+ private QuerySchedulingInfo pollScheduledQuery() {
+ if (pool.size() > 1) {
+ Collections.sort(pool, COMPARATOR);
+ }
+ return pool.poll();
+ }
+
+ private void wakeupProcessor() {
+ synchronized (queryProcessor) {
+ queryProcessor.notifyAll();
+ }
+ }
+
+ private final class QueryProcessor implements Runnable {
+ @Override
+ public void run() {
+
+ QuerySchedulingInfo query;
+
+ while (!stopped.get() && !Thread.currentThread().isInterrupted()) {
+ query = null;
+ if (getRunningQueries().size() == 0) {
+ query = pollScheduledQuery();
+ }
+
+ if (query != null) {
+ try {
+ manager.startQueryJob(query.getQueryId());
+ } catch (Throwable t) {
+ LOG.fatal("Exception during query startup:", t);
+ manager.stopQuery(query.getQueryId());
+ }
+ }
+
+ synchronized (queryProcessor) {
+ try {
+ queryProcessor.wait(500);
+ } catch (InterruptedException e) {
+ if (stopped.get()) {
+ break;
+ }
+ LOG.warn("Exception during shutdown: ", e);
+ }
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/session/InvalidSessionException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/InvalidSessionException.java b/tajo-core/src/main/java/org/apache/tajo/master/session/InvalidSessionException.java
deleted file mode 100644
index 3f48ca5..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/session/InvalidSessionException.java
+++ /dev/null
@@ -1,25 +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.tajo.master.session;
-
-public class InvalidSessionException extends Exception {
- public InvalidSessionException(String sessionId) {
- super("Invalid session id \"" + sessionId + "\"");
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/session/NoSuchSessionVariableException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/NoSuchSessionVariableException.java b/tajo-core/src/main/java/org/apache/tajo/master/session/NoSuchSessionVariableException.java
deleted file mode 100644
index 686d860..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/session/NoSuchSessionVariableException.java
+++ /dev/null
@@ -1,25 +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.tajo.master.session;
-
-public class NoSuchSessionVariableException extends Exception {
- public NoSuchSessionVariableException(String varname) {
- super("No such session variable \"" + varname + "\"");
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/session/Session.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/Session.java b/tajo-core/src/main/java/org/apache/tajo/master/session/Session.java
deleted file mode 100644
index 5f44ecb..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/session/Session.java
+++ /dev/null
@@ -1,196 +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.tajo.master.session;
-
-import com.google.common.collect.ImmutableMap;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.SessionVars;
-import org.apache.tajo.master.NonForwardQueryResultScanner;
-import org.apache.tajo.util.KeyValueSet;
-import org.apache.tajo.common.ProtoObject;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.apache.tajo.ipc.TajoWorkerProtocol.SessionProto;
-
-public class Session implements SessionConstants, ProtoObject<SessionProto>, Cloneable {
- private static final Log LOG = LogFactory.getLog(Session.class);
-
- private final String sessionId;
- private final String userName;
- private String currentDatabase;
- private final Map<String, String> sessionVariables;
- private final Map<QueryId, NonForwardQueryResultScanner> nonForwardQueryMap = new HashMap<QueryId, NonForwardQueryResultScanner>();
-
- // transient status
- private volatile long lastAccessTime;
-
- public Session(String sessionId, String userName, String databaseName) {
- this.sessionId = sessionId;
- this.userName = userName;
- this.currentDatabase = databaseName;
- this.lastAccessTime = System.currentTimeMillis();
-
- this.sessionVariables = new HashMap<String, String>();
- sessionVariables.put(SessionVars.SESSION_ID.keyname(), sessionId);
- sessionVariables.put(SessionVars.USERNAME.keyname(), userName);
- selectDatabase(databaseName);
- }
-
- public Session(SessionProto proto) {
- sessionId = proto.getSessionId();
- userName = proto.getUsername();
- currentDatabase = proto.getCurrentDatabase();
- lastAccessTime = proto.getLastAccessTime();
- KeyValueSet keyValueSet = new KeyValueSet(proto.getVariables());
- sessionVariables = keyValueSet.getAllKeyValus();
- }
-
- public String getSessionId() {
- return sessionId;
- }
-
- public String getUserName() {
- return userName;
- }
-
- public void updateLastAccessTime() {
- lastAccessTime = System.currentTimeMillis();
- }
-
- public long getLastAccessTime() {
- return lastAccessTime;
- }
-
- public void setVariable(String name, String value) {
- synchronized (sessionVariables) {
- sessionVariables.put(SessionVars.handleDeprecatedName(name), value);
- }
- }
-
- public String getVariable(String name) throws NoSuchSessionVariableException {
- synchronized (sessionVariables) {
- if (sessionVariables.containsKey(name)) {
- return sessionVariables.get(SessionVars.handleDeprecatedName(name));
- } else {
- throw new NoSuchSessionVariableException(name);
- }
- }
- }
-
- public void removeVariable(String name) {
- synchronized (sessionVariables) {
- sessionVariables.remove(SessionVars.handleDeprecatedName(name));
- }
- }
-
- public synchronized Map<String, String> getAllVariables() {
- synchronized (sessionVariables) {
- sessionVariables.put(SessionVars.SESSION_ID.keyname(), sessionId);
- sessionVariables.put(SessionVars.USERNAME.keyname(), userName);
- sessionVariables.put(SessionVars.SESSION_LAST_ACCESS_TIME.keyname(), String.valueOf(lastAccessTime));
- sessionVariables.put(SessionVars.CURRENT_DATABASE.keyname(), currentDatabase);
- return ImmutableMap.copyOf(sessionVariables);
- }
- }
-
- public synchronized void selectDatabase(String databaseName) {
- this.currentDatabase = databaseName;
- }
-
- public synchronized String getCurrentDatabase() {
- return currentDatabase;
- }
-
- @Override
- public SessionProto getProto() {
- SessionProto.Builder builder = SessionProto.newBuilder();
- builder.setSessionId(getSessionId());
- builder.setUsername(getUserName());
- builder.setCurrentDatabase(getCurrentDatabase());
- builder.setLastAccessTime(lastAccessTime);
- KeyValueSet variables = new KeyValueSet();
-
- synchronized (sessionVariables) {
- variables.putAll(this.sessionVariables);
- builder.setVariables(variables.getProto());
- return builder.build();
- }
- }
-
- public String toString() {
- return "user=" + getUserName() + ",id=" + getSessionId() +",last_atime=" + getLastAccessTime();
- }
-
- public Session clone() throws CloneNotSupportedException {
- Session newSession = (Session) super.clone();
- newSession.sessionVariables.putAll(getAllVariables());
- return newSession;
- }
-
- public NonForwardQueryResultScanner getNonForwardQueryResultScanner(QueryId queryId) {
- synchronized (nonForwardQueryMap) {
- return nonForwardQueryMap.get(queryId);
- }
- }
-
- public void addNonForwardQueryResultScanner(NonForwardQueryResultScanner resultScanner) {
- synchronized (nonForwardQueryMap) {
- nonForwardQueryMap.put(resultScanner.getQueryId(), resultScanner);
- }
- }
-
- public void closeNonForwardQueryResultScanner(QueryId queryId) {
- NonForwardQueryResultScanner resultScanner;
- synchronized (nonForwardQueryMap) {
- resultScanner = nonForwardQueryMap.remove(queryId);
- }
-
- if (resultScanner != null) {
- try {
- resultScanner.close();
- } catch (Exception e) {
- LOG.error("NonForwardQueryResultScanne close error: " + e.getMessage(), e);
- }
- }
- }
-
- public void close() {
- try {
- synchronized (nonForwardQueryMap) {
- for (NonForwardQueryResultScanner eachQueryScanner: nonForwardQueryMap.values()) {
- try {
- eachQueryScanner.close();
- } catch (Exception e) {
- LOG.error("Error while closing NonForwardQueryResultScanner: " +
- eachQueryScanner.getSessionId() + ", " + e.getMessage(), e);
- }
- }
-
- nonForwardQueryMap.clear();
- }
- } catch (Throwable t) {
- LOG.error(t.getMessage(), t);
- throw new RuntimeException(t.getMessage(), t);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/session/SessionConstants.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionConstants.java b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionConstants.java
deleted file mode 100644
index 46f49a2..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionConstants.java
+++ /dev/null
@@ -1,23 +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.tajo.master.session;
-
-public interface SessionConstants {
-
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEvent.java
deleted file mode 100644
index dce3ba6..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEvent.java
+++ /dev/null
@@ -1,34 +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.tajo.master.session;
-
-import org.apache.hadoop.yarn.event.AbstractEvent;
-
-public class SessionEvent extends AbstractEvent<SessionEventType> {
- private final String sessionId;
-
- public SessionEvent(String sessionId, SessionEventType sessionEventType) {
- super(sessionEventType);
- this.sessionId = sessionId;
- }
-
- public String getSessionId() {
- return sessionId;
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEventType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEventType.java b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEventType.java
deleted file mode 100644
index 64c6fc6..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionEventType.java
+++ /dev/null
@@ -1,24 +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.tajo.master.session;
-
-public enum SessionEventType {
- EXPIRE,
- PING
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/session/SessionLivelinessMonitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionLivelinessMonitor.java b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionLivelinessMonitor.java
deleted file mode 100644
index 912f769..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionLivelinessMonitor.java
+++ /dev/null
@@ -1,53 +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.tajo.master.session;
-
-import com.google.common.base.Preconditions;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.event.Dispatcher;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.util.AbstractLivelinessMonitor;
-import org.apache.hadoop.yarn.util.SystemClock;
-import org.apache.tajo.conf.TajoConf;
-
-public class SessionLivelinessMonitor extends AbstractLivelinessMonitor<String> {
-
- private EventHandler dispatcher;
-
- public SessionLivelinessMonitor(Dispatcher d) {
- super(SessionLivelinessMonitor.class.getSimpleName(), new SystemClock());
- this.dispatcher = d.getEventHandler();
- }
-
- public void serviceInit(Configuration conf) throws Exception {
- Preconditions.checkArgument(conf instanceof TajoConf);
- TajoConf systemConf = (TajoConf) conf;
-
- // seconds
- int expireIntvl = systemConf.getIntVar(TajoConf.ConfVars.$CLIENT_SESSION_EXPIRY_TIME);
- setExpireInterval(expireIntvl);
- setMonitorInterval(expireIntvl / 3);
- super.serviceInit(conf);
- }
-
- @Override
- protected void expire(String id) {
- dispatcher.handle(new SessionEvent(id, SessionEventType.EXPIRE));
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/session/SessionManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionManager.java b/tajo-core/src/main/java/org/apache/tajo/master/session/SessionManager.java
deleted file mode 100644
index d701d03..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/session/SessionManager.java
+++ /dev/null
@@ -1,144 +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.tajo.master.session;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.CompositeService;
-import org.apache.hadoop.yarn.event.Dispatcher;
-import org.apache.hadoop.yarn.event.EventHandler;
-
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-
-public class SessionManager extends CompositeService implements EventHandler<SessionEvent> {
- private static final Log LOG = LogFactory.getLog(SessionManager.class);
-
- public final ConcurrentHashMap<String, Session> sessions = new ConcurrentHashMap<String, Session>();
- private final Dispatcher dispatcher;
- private SessionLivelinessMonitor sessionLivelinessMonitor;
-
-
- public SessionManager(Dispatcher dispatcher) {
- super(SessionManager.class.getSimpleName());
- this.dispatcher = dispatcher;
- }
-
- @Override
- public void serviceInit(Configuration conf) throws Exception {
- sessionLivelinessMonitor = new SessionLivelinessMonitor(dispatcher);
- addIfService(sessionLivelinessMonitor);
- super.serviceInit(conf);
- }
-
- @Override
- public void serviceStop() throws Exception {
- super.serviceStop();
- }
-
- private void assertSessionExistence(String sessionId) throws InvalidSessionException {
- if (!sessions.containsKey(sessionId)) {
- throw new InvalidSessionException(sessionId);
- }
- }
-
- public String createSession(String username, String baseDatabaseName) throws InvalidSessionException {
- String sessionId;
- Session oldSession;
-
- sessionId = UUID.randomUUID().toString();
- Session newSession = new Session(sessionId, username, baseDatabaseName);
- oldSession = sessions.putIfAbsent(sessionId, newSession);
- if (oldSession != null) {
- throw new InvalidSessionException("Session id is duplicated: " + oldSession.getSessionId());
- }
- LOG.info("Session " + sessionId + " is created." );
- return sessionId;
- }
-
- public Session removeSession(String sessionId) {
- if (sessions.containsKey(sessionId)) {
- LOG.info("Session " + sessionId + " is removed.");
- Session session = sessions.remove(sessionId);
- session.close();
- return session;
- } else {
- LOG.error("No such session id: " + sessionId);
- return null;
- }
- }
-
- public Session getSession(String sessionId) throws InvalidSessionException {
- assertSessionExistence(sessionId);
- touch(sessionId);
- return sessions.get(sessionId);
- }
-
- public void setVariable(String sessionId, String name, String value) throws InvalidSessionException {
- assertSessionExistence(sessionId);
- touch(sessionId);
- sessions.get(sessionId).setVariable(name, value);
- }
-
- public String getVariable(String sessionId, String name)
- throws InvalidSessionException, NoSuchSessionVariableException {
- assertSessionExistence(sessionId);
- touch(sessionId);
- return sessions.get(sessionId).getVariable(name);
- }
-
- public void removeVariable(String sessionId, String name) throws InvalidSessionException {
- assertSessionExistence(sessionId);
- touch(sessionId);
- sessions.get(sessionId).removeVariable(name);
- }
-
- public Map<String, String> getAllVariables(String sessionId) throws InvalidSessionException {
- assertSessionExistence(sessionId);
- touch(sessionId);
- return sessions.get(sessionId).getAllVariables();
- }
-
- public void touch(String sessionId) throws InvalidSessionException {
- assertSessionExistence(sessionId);
- sessions.get(sessionId).updateLastAccessTime();
- sessionLivelinessMonitor.receivedPing(sessionId);
- }
-
- @Override
- public void handle(SessionEvent event) {
- LOG.info("Processing " + event.getSessionId() + " of type " + event.getType());
-
- try {
- assertSessionExistence(event.getSessionId());
- touch(event.getSessionId());
- } catch (InvalidSessionException e) {
- LOG.error(e);
- }
-
- if (event.getType() == SessionEventType.EXPIRE) {
- Session session = removeSession(event.getSessionId());
- if (session != null) {
- LOG.info("[Expired] Session username=" + session.getUserName() + ",sessionid=" + event.getSessionId());
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/metrics/CatalogMetricsGaugeSet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/metrics/CatalogMetricsGaugeSet.java b/tajo-core/src/main/java/org/apache/tajo/metrics/CatalogMetricsGaugeSet.java
new file mode 100644
index 0000000..82ebe29
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/metrics/CatalogMetricsGaugeSet.java
@@ -0,0 +1,56 @@
+/**
+ * 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.tajo.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricSet;
+import org.apache.tajo.master.TajoMaster;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+
+public class CatalogMetricsGaugeSet implements MetricSet {
+ TajoMaster.MasterContext tajoMasterContext;
+ public CatalogMetricsGaugeSet(TajoMaster.MasterContext tajoMasterContext) {
+ this.tajoMasterContext = tajoMasterContext;
+ }
+
+ @Override
+ public Map<String, Metric> getMetrics() {
+ Map<String, Metric> metricsMap = new HashMap<String, Metric>();
+ metricsMap.put("numTables", new Gauge<Integer>() {
+ @Override
+ public Integer getValue() {
+ return tajoMasterContext.getCatalog().getAllTableNames(DEFAULT_DATABASE_NAME).size();
+ }
+ });
+
+ metricsMap.put("numFunctions", new Gauge<Integer>() {
+ @Override
+ public Integer getValue() {
+ return tajoMasterContext.getCatalog().getFunctions().size();
+ }
+ });
+
+ return metricsMap;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/metrics/WorkerResourceMetricsGaugeSet.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/metrics/WorkerResourceMetricsGaugeSet.java b/tajo-core/src/main/java/org/apache/tajo/metrics/WorkerResourceMetricsGaugeSet.java
new file mode 100644
index 0000000..229a80a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/metrics/WorkerResourceMetricsGaugeSet.java
@@ -0,0 +1,74 @@
+/**
+ * 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.tajo.metrics;
+
+import com.codahale.metrics.Gauge;
+import com.codahale.metrics.Metric;
+import com.codahale.metrics.MetricSet;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.rm.Worker;
+import org.apache.tajo.master.rm.WorkerState;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class WorkerResourceMetricsGaugeSet implements MetricSet {
+ TajoMaster.MasterContext tajoMasterContext;
+ public WorkerResourceMetricsGaugeSet(TajoMaster.MasterContext tajoMasterContext) {
+ this.tajoMasterContext = tajoMasterContext;
+ }
+
+ @Override
+ public Map<String, Metric> getMetrics() {
+ Map<String, Metric> metricsMap = new HashMap<String, Metric>();
+ metricsMap.put("totalWorkers", new Gauge<Integer>() {
+ @Override
+ public Integer getValue() {
+ return tajoMasterContext.getResourceManager().getWorkers().size();
+ }
+ });
+
+ metricsMap.put("liveWorkers", new Gauge<Integer>() {
+ @Override
+ public Integer getValue() {
+ return getNumWorkers(WorkerState.RUNNING);
+ }
+ });
+
+ metricsMap.put("deadWorkers", new Gauge<Integer>() {
+ @Override
+ public Integer getValue() {
+ return getNumWorkers(WorkerState.LOST);
+ }
+ });
+
+ return metricsMap;
+ }
+
+ protected int getNumWorkers(WorkerState status) {
+ int numWorkers = 0;
+ for(Worker eachWorker: tajoMasterContext.getResourceManager().getWorkers().values()) {
+ if(eachWorker.getState() == status) {
+ numWorkers++;
+ }
+ }
+
+ return numWorkers;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java
new file mode 100644
index 0000000..e45f274
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/AbstractTaskScheduler.java
@@ -0,0 +1,56 @@
+/**
+ * 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.tajo.querymaster;
+
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tajo.master.event.TaskRequestEvent;
+import org.apache.tajo.master.event.TaskSchedulerEvent;
+
+
+public abstract class AbstractTaskScheduler extends AbstractService implements EventHandler<TaskSchedulerEvent> {
+
+ protected int hostLocalAssigned;
+ protected int rackLocalAssigned;
+ protected int totalAssigned;
+
+ /**
+ * Construct the service.
+ *
+ * @param name service name
+ */
+ public AbstractTaskScheduler(String name) {
+ super(name);
+ }
+
+ public int getHostLocalAssigned() {
+ return hostLocalAssigned;
+ }
+
+ public int getRackLocalAssigned() {
+ return rackLocalAssigned;
+ }
+
+ public int getTotalAssigned() {
+ return totalAssigned;
+ }
+
+ public abstract void handleTaskRequestEvent(TaskRequestEvent event);
+ public abstract int remainingScheduledObjectNum();
+}
[02/16] tajo git commit: TAJO-1288: Refactoring
org.apache.tajo.master package.
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java b/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java
new file mode 100644
index 0000000..a6f9f74
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/ha/TestHAServiceHDFSImpl.java
@@ -0,0 +1,153 @@
+/**
+ * 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.tajo.ha;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.client.TajoClientImpl;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.master.TajoMaster;
+import org.junit.Test;
+
+import static junit.framework.TestCase.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestHAServiceHDFSImpl {
+ private static Log LOG = LogFactory.getLog(TestHAServiceHDFSImpl.class);
+
+ private TajoTestingCluster cluster;
+ private TajoMaster backupMaster;
+
+ private TajoConf conf;
+ private TajoClient client;
+
+ private Path haPath, activePath, backupPath;
+
+ private String masterAddress;
+
+ @Test
+ public final void testAutoFailOver() throws Exception {
+ cluster = new TajoTestingCluster(true);
+
+ cluster.startMiniCluster(1);
+ conf = cluster.getConfiguration();
+ client = new TajoClientImpl(conf);
+
+ try {
+ FileSystem fs = cluster.getDefaultFileSystem();
+
+ masterAddress = HAServiceUtil.getMasterUmbilicalName(conf).split(":")[0];
+
+ setConfiguration();
+
+ backupMaster = new TajoMaster();
+ backupMaster.init(conf);
+ backupMaster.start();
+
+ assertNotEquals(cluster.getMaster().getMasterName(), backupMaster.getMasterName());
+
+ verifySystemDirectories(fs);
+
+ Path backupMasterFile = new Path(backupPath, backupMaster.getMasterName()
+ .replaceAll(":", "_"));
+ assertTrue(fs.exists(backupMasterFile));
+
+ assertTrue(cluster.getMaster().isActiveMaster());
+ assertFalse(backupMaster.isActiveMaster());
+
+ createDatabaseAndTable();
+ verifyDataBaseAndTable();
+ client.close();
+
+ cluster.getMaster().stop();
+
+ Thread.sleep(7000);
+
+ assertFalse(cluster.getMaster().isActiveMaster());
+ assertTrue(backupMaster.isActiveMaster());
+
+ client = new TajoClientImpl(conf);
+ verifyDataBaseAndTable();
+ } finally {
+ client.close();
+ backupMaster.stop();
+ cluster.shutdownMiniCluster();
+ }
+ }
+
+ private void setConfiguration() {
+ conf = cluster.getConfiguration();
+
+ conf.setVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS,
+ masterAddress + ":" + NetUtils.getFreeSocketPort());
+ conf.setVar(TajoConf.ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS,
+ masterAddress + ":" + NetUtils.getFreeSocketPort());
+ conf.setVar(TajoConf.ConfVars.RESOURCE_TRACKER_RPC_ADDRESS,
+ masterAddress + ":" + NetUtils.getFreeSocketPort());
+ conf.setVar(TajoConf.ConfVars.CATALOG_ADDRESS,
+ masterAddress + ":" + NetUtils.getFreeSocketPort());
+ conf.setVar(TajoConf.ConfVars.TAJO_MASTER_INFO_ADDRESS,
+ masterAddress + ":" + NetUtils.getFreeSocketPort());
+ conf.setBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE, true);
+
+ //Client API service RPC Server
+ conf.setIntVar(TajoConf.ConfVars.MASTER_SERVICE_RPC_SERVER_WORKER_THREAD_NUM, 2);
+ conf.setIntVar(TajoConf.ConfVars.WORKER_SERVICE_RPC_SERVER_WORKER_THREAD_NUM, 2);
+
+ // Internal RPC Server
+ conf.setIntVar(TajoConf.ConfVars.MASTER_RPC_SERVER_WORKER_THREAD_NUM, 2);
+ conf.setIntVar(TajoConf.ConfVars.QUERY_MASTER_RPC_SERVER_WORKER_THREAD_NUM, 2);
+ conf.setIntVar(TajoConf.ConfVars.WORKER_RPC_SERVER_WORKER_THREAD_NUM, 2);
+ conf.setIntVar(TajoConf.ConfVars.CATALOG_RPC_SERVER_WORKER_THREAD_NUM, 2);
+ conf.setIntVar(TajoConf.ConfVars.SHUFFLE_RPC_SERVER_WORKER_THREAD_NUM, 2);
+ }
+
+ private void verifySystemDirectories(FileSystem fs) throws Exception {
+ haPath = TajoConf.getSystemHADir(cluster.getConfiguration());
+ assertTrue(fs.exists(haPath));
+
+ activePath = new Path(haPath, TajoConstants.SYSTEM_HA_ACTIVE_DIR_NAME);
+ assertTrue(fs.exists(activePath));
+
+ backupPath = new Path(haPath, TajoConstants.SYSTEM_HA_BACKUP_DIR_NAME);
+ assertTrue(fs.exists(backupPath));
+
+ assertEquals(1, fs.listStatus(activePath).length);
+ assertEquals(1, fs.listStatus(backupPath).length);
+ }
+
+ private void createDatabaseAndTable() throws Exception {
+ client.executeQuery("CREATE TABLE default.table1 (age int);");
+ client.executeQuery("CREATE TABLE default.table2 (age int);");
+ }
+
+ private void verifyDataBaseAndTable() throws Exception {
+ client.existDatabase("default");
+ client.existTable("default.table1");
+ client.existTable("default.table2");
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java b/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java
index bdd6dfc..fa7fdf0 100644
--- a/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/master/TestNonForwardQueryResultSystemScanner.java
@@ -40,6 +40,8 @@ import org.apache.tajo.datum.Datum;
import org.apache.tajo.engine.parser.SQLAnalyzer;
import org.apache.tajo.engine.query.QueryContext;
import org.apache.tajo.master.TajoMaster.MasterContext;
+import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
+import org.apache.tajo.master.exec.NonForwardQueryResultSystemScanner;
import org.apache.tajo.plan.LogicalOptimizer;
import org.apache.tajo.plan.LogicalPlan;
import org.apache.tajo.plan.LogicalPlanner;
@@ -188,7 +190,7 @@ public class TestNonForwardQueryResultSystemScanner {
}
NonForwardQueryResultScanner queryResultScanner =
- new NonForwardQueryResultSystemScanner(masterContext, logicalPlan, queryId,
+ new NonForwardQueryResultSystemScanner(masterContext, logicalPlan, queryId,
sessionId, maxRow);
return queryResultScanner;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java b/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java
index 66d74c4..438867e 100644
--- a/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/master/TestRepartitioner.java
@@ -25,9 +25,9 @@ import org.apache.tajo.LocalTajoTestingUtility;
import org.apache.tajo.QueryId;
import org.apache.tajo.TestTajoIds;
import org.apache.tajo.ipc.TajoWorkerProtocol;
-import org.apache.tajo.master.querymaster.Task;
-import org.apache.tajo.master.querymaster.Task.IntermediateEntry;
-import org.apache.tajo.master.querymaster.Repartitioner;
+import org.apache.tajo.querymaster.Task;
+import org.apache.tajo.querymaster.Task.IntermediateEntry;
+import org.apache.tajo.querymaster.Repartitioner;
import org.apache.tajo.util.Pair;
import org.apache.tajo.util.TUtil;
import org.apache.tajo.worker.FetchImpl;
@@ -38,7 +38,7 @@ import java.net.URI;
import java.util.*;
import static junit.framework.Assert.assertEquals;
-import static org.apache.tajo.master.querymaster.Repartitioner.FetchGroupMeta;
+import static org.apache.tajo.querymaster.Repartitioner.FetchGroupMeta;
import static org.apache.tajo.plan.serder.PlanProto.ShuffleType;
import static org.apache.tajo.plan.serder.PlanProto.ShuffleType.HASH_SHUFFLE;
import static org.apache.tajo.plan.serder.PlanProto.ShuffleType.SCATTERED_HASH_SHUFFLE;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/master/ha/TestHAServiceHDFSImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/ha/TestHAServiceHDFSImpl.java b/tajo-core/src/test/java/org/apache/tajo/master/ha/TestHAServiceHDFSImpl.java
deleted file mode 100644
index e1806e1..0000000
--- a/tajo-core/src/test/java/org/apache/tajo/master/ha/TestHAServiceHDFSImpl.java
+++ /dev/null
@@ -1,158 +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.tajo.master.ha;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.tajo.TajoConstants;
-import org.apache.tajo.TajoTestingCluster;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.client.TajoClient;
-import org.apache.tajo.client.TajoClientImpl;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.ha.HAServiceUtil;
-import org.apache.tajo.master.TajoMaster;
-import org.junit.Test;
-
-import java.util.List;
-
-import static junit.framework.TestCase.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestHAServiceHDFSImpl {
- private static Log LOG = LogFactory.getLog(TestHAServiceHDFSImpl.class);
-
- private TajoTestingCluster cluster;
- private TajoMaster backupMaster;
-
- private TajoConf conf;
- private TajoClient client;
-
- private Path haPath, activePath, backupPath;
-
- private String masterAddress;
-
- @Test
- public final void testAutoFailOver() throws Exception {
- cluster = new TajoTestingCluster(true);
-
- cluster.startMiniCluster(1);
- conf = cluster.getConfiguration();
- client = new TajoClientImpl(conf);
-
- try {
- FileSystem fs = cluster.getDefaultFileSystem();
-
- masterAddress = HAServiceUtil.getMasterUmbilicalName(conf).split(":")[0];
-
- setConfiguration();
-
- backupMaster = new TajoMaster();
- backupMaster.init(conf);
- backupMaster.start();
-
- assertNotEquals(cluster.getMaster().getMasterName(), backupMaster.getMasterName());
-
- verifySystemDirectories(fs);
-
- Path backupMasterFile = new Path(backupPath, backupMaster.getMasterName()
- .replaceAll(":", "_"));
- assertTrue(fs.exists(backupMasterFile));
-
- assertTrue(cluster.getMaster().isActiveMaster());
- assertFalse(backupMaster.isActiveMaster());
-
- createDatabaseAndTable();
- verifyDataBaseAndTable();
- client.close();
-
- cluster.getMaster().stop();
-
- Thread.sleep(7000);
-
- assertFalse(cluster.getMaster().isActiveMaster());
- assertTrue(backupMaster.isActiveMaster());
-
- client = new TajoClientImpl(conf);
- verifyDataBaseAndTable();
- } finally {
- client.close();
- backupMaster.stop();
- cluster.shutdownMiniCluster();
- }
- }
-
- private void setConfiguration() {
- conf = cluster.getConfiguration();
-
- conf.setVar(TajoConf.ConfVars.TAJO_MASTER_CLIENT_RPC_ADDRESS,
- masterAddress + ":" + NetUtils.getFreeSocketPort());
- conf.setVar(TajoConf.ConfVars.TAJO_MASTER_UMBILICAL_RPC_ADDRESS,
- masterAddress + ":" + NetUtils.getFreeSocketPort());
- conf.setVar(TajoConf.ConfVars.RESOURCE_TRACKER_RPC_ADDRESS,
- masterAddress + ":" + NetUtils.getFreeSocketPort());
- conf.setVar(TajoConf.ConfVars.CATALOG_ADDRESS,
- masterAddress + ":" + NetUtils.getFreeSocketPort());
- conf.setVar(TajoConf.ConfVars.TAJO_MASTER_INFO_ADDRESS,
- masterAddress + ":" + NetUtils.getFreeSocketPort());
- conf.setBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE, true);
-
- //Client API service RPC Server
- conf.setIntVar(TajoConf.ConfVars.MASTER_SERVICE_RPC_SERVER_WORKER_THREAD_NUM, 2);
- conf.setIntVar(TajoConf.ConfVars.WORKER_SERVICE_RPC_SERVER_WORKER_THREAD_NUM, 2);
-
- // Internal RPC Server
- conf.setIntVar(TajoConf.ConfVars.MASTER_RPC_SERVER_WORKER_THREAD_NUM, 2);
- conf.setIntVar(TajoConf.ConfVars.QUERY_MASTER_RPC_SERVER_WORKER_THREAD_NUM, 2);
- conf.setIntVar(TajoConf.ConfVars.WORKER_RPC_SERVER_WORKER_THREAD_NUM, 2);
- conf.setIntVar(TajoConf.ConfVars.CATALOG_RPC_SERVER_WORKER_THREAD_NUM, 2);
- conf.setIntVar(TajoConf.ConfVars.SHUFFLE_RPC_SERVER_WORKER_THREAD_NUM, 2);
- }
-
- private void verifySystemDirectories(FileSystem fs) throws Exception {
- haPath = TajoConf.getSystemHADir(cluster.getConfiguration());
- assertTrue(fs.exists(haPath));
-
- activePath = new Path(haPath, TajoConstants.SYSTEM_HA_ACTIVE_DIR_NAME);
- assertTrue(fs.exists(activePath));
-
- backupPath = new Path(haPath, TajoConstants.SYSTEM_HA_BACKUP_DIR_NAME);
- assertTrue(fs.exists(backupPath));
-
- assertEquals(1, fs.listStatus(activePath).length);
- assertEquals(1, fs.listStatus(backupPath).length);
- }
-
- private void createDatabaseAndTable() throws Exception {
- client.executeQuery("CREATE TABLE default.table1 (age int);");
- client.executeQuery("CREATE TABLE default.table2 (age int);");
- }
-
- private void verifyDataBaseAndTable() throws Exception {
- client.existDatabase("default");
- client.existTable("default.table1");
- client.existTable("default.table2");
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestIntermediateEntry.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestIntermediateEntry.java b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestIntermediateEntry.java
deleted file mode 100644
index 7698987..0000000
--- a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestIntermediateEntry.java
+++ /dev/null
@@ -1,53 +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.tajo.master.querymaster;
-
-import org.apache.tajo.util.Pair;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestIntermediateEntry {
- @Test
- public void testPage() {
- Task.IntermediateEntry interm = new Task.IntermediateEntry(-1, -1, 1, null);
-
- List<Pair<Long, Integer>> pages = new ArrayList<Pair<Long, Integer>>();
- pages.add(new Pair(0L, 1441275));
- pages.add(new Pair(1441275L, 1447446));
- pages.add(new Pair(2888721L, 1442507));
-
- interm.setPages(pages);
-
- long splitBytes = 3 * 1024 * 1024;
-
- List<Pair<Long, Long>> splits = interm.split(splitBytes, splitBytes);
- assertEquals(2, splits.size());
-
- long[][] expected = { {0, 1441275 + 1447446}, {1441275 + 1447446, 1442507} };
- for (int i = 0; i < 2; i++) {
- Pair<Long, Long> eachSplit = splits.get(i);
- assertEquals(expected[i][0], eachSplit.getFirst().longValue());
- assertEquals(expected[i][1], eachSplit.getSecond().longValue());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestKillQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestKillQuery.java
deleted file mode 100644
index 8ca4cff..0000000
--- a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestKillQuery.java
+++ /dev/null
@@ -1,125 +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.tajo.master.querymaster;
-
-import org.apache.tajo.*;
-import org.apache.tajo.algebra.Expr;
-import org.apache.tajo.benchmark.TPCH;
-import org.apache.tajo.catalog.CatalogService;
-import org.apache.tajo.client.TajoClient;
-import org.apache.tajo.client.TajoClientImpl;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.parser.SQLAnalyzer;
-import org.apache.tajo.engine.planner.global.GlobalPlanner;
-import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.master.event.QueryEvent;
-import org.apache.tajo.master.event.QueryEventType;
-import org.apache.tajo.master.session.Session;
-import org.apache.tajo.plan.LogicalOptimizer;
-import org.apache.tajo.plan.LogicalPlan;
-import org.apache.tajo.plan.LogicalPlanner;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-
-import static org.junit.Assert.*;
-
-public class TestKillQuery {
- private static TajoTestingCluster cluster;
- private static TajoConf conf;
- private static TajoClient client;
-
- @BeforeClass
- public static void setUp() throws Exception {
- cluster = new TajoTestingCluster();
- cluster.startMiniClusterInLocal(1);
- conf = cluster.getConfiguration();
- client = new TajoClientImpl(cluster.getConfiguration());
- File file = TPCH.getDataFile("lineitem");
- client.executeQueryAndGetResult("create external table default.lineitem (l_orderkey int, l_partkey int) "
- + "using text location 'file://" + file.getAbsolutePath() + "'");
- assertTrue(client.existTable("default.lineitem"));
- }
-
- @AfterClass
- public static void tearDown() throws IOException {
- if (client != null) client.close();
- if (cluster != null) cluster.shutdownMiniCluster();
- }
-
- @Test
- public final void testKillQueryFromInitState() throws Exception {
- SQLAnalyzer analyzer = new SQLAnalyzer();
- QueryContext defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
- Session session = LocalTajoTestingUtility.createDummySession();
- CatalogService catalog = cluster.getMaster().getCatalog();
- String query = "select l_orderkey, l_partkey from lineitem group by l_orderkey, l_partkey order by l_orderkey";
-
- LogicalPlanner planner = new LogicalPlanner(catalog);
- LogicalOptimizer optimizer = new LogicalOptimizer(conf);
- Expr expr = analyzer.parse(query);
- LogicalPlan plan = planner.createPlan(defaultContext, expr);
-
- optimizer.optimize(plan);
-
- QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
- QueryContext queryContext = new QueryContext(conf);
- MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
- GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
- globalPlanner.build(masterPlan);
-
- QueryMaster qm = cluster.getTajoWorkers().get(0).getWorkerContext().getQueryMaster();
- QueryMasterTask queryMasterTask = new QueryMasterTask(qm.getContext(),
- queryId, session, defaultContext, expr.toJson(), plan.getRootBlock().getRoot().toJson());
-
- queryMasterTask.init(conf);
- queryMasterTask.getQueryTaskContext().getDispatcher().start();
- queryMasterTask.startQuery();
-
- try{
- cluster.waitForQueryState(queryMasterTask.getQuery(), TajoProtos.QueryState.QUERY_RUNNING, 2);
- } finally {
- assertEquals(TajoProtos.QueryState.QUERY_RUNNING, queryMasterTask.getQuery().getSynchronizedState());
- }
-
- Stage stage = queryMasterTask.getQuery().getStages().iterator().next();
- assertNotNull(stage);
-
- try{
- cluster.waitForStageState(stage, StageState.INITED, 2);
- } finally {
- assertEquals(StageState.INITED, stage.getSynchronizedState());
- }
-
- // fire kill event
- Query q = queryMasterTask.getQuery();
- q.handle(new QueryEvent(queryId, QueryEventType.KILL));
-
- try{
- cluster.waitForQueryState(queryMasterTask.getQuery(), TajoProtos.QueryState.QUERY_KILLED, 50);
- } finally {
- assertEquals(TajoProtos.QueryState.QUERY_KILLED, queryMasterTask.getQuery().getSynchronizedState());
- }
- queryMasterTask.stop();
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryProgress.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryProgress.java b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryProgress.java
deleted file mode 100644
index 4a6ca00..0000000
--- a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryProgress.java
+++ /dev/null
@@ -1,75 +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.tajo.master.querymaster;
-
-import org.apache.tajo.*;
-import org.apache.tajo.client.QueryStatus;
-import org.apache.tajo.client.TajoClient;
-import org.apache.tajo.client.TajoClientImpl;
-import org.apache.tajo.client.TajoClientUtil;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.ipc.ClientProtos;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import static org.junit.Assert.*;
-
-@Category(IntegrationTest.class)
-public class TestQueryProgress {
- private static TajoTestingCluster cluster;
- private static TajoConf conf;
- private static TajoClient client;
-
- @BeforeClass
- public static void setUp() throws Exception {
- cluster = TpchTestBase.getInstance().getTestingCluster();
- conf = cluster.getConfiguration();
- client = new TajoClientImpl(conf);
- }
-
- @AfterClass
- public static void tearDown() throws Exception {
- client.close();
- }
-
- @Test(timeout = 10000)
- public final void testQueryProgress() throws Exception {
- ClientProtos.SubmitQueryResponse res = client.executeQuery("select l_orderkey from lineitem group by l_orderkey");
- QueryId queryId = new QueryId(res.getQueryId());
-
- float prevProgress = 0;
- while (true) {
- QueryStatus status = client.getQueryStatus(queryId);
- if (status == null) continue;
-
- float progress = status.getProgress();
-
- if (prevProgress > progress) {
- fail("Previous progress: " + prevProgress + ", Current progress : " + progress);
- }
- prevProgress = progress;
- assertTrue(progress <= 1.0f);
-
- if (TajoClientUtil.isQueryComplete(status.getState())) break;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestTaskStatusUpdate.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestTaskStatusUpdate.java b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestTaskStatusUpdate.java
deleted file mode 100644
index 3a54478..0000000
--- a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestTaskStatusUpdate.java
+++ /dev/null
@@ -1,194 +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.tajo.master.querymaster;
-
-import org.apache.tajo.IntegrationTest;
-import org.apache.tajo.QueryTestCaseBase;
-import org.apache.tajo.TajoConstants;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.worker.TajoWorker;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import java.sql.ResultSet;
-import java.util.*;
-
-import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
-import static org.junit.Assert.*;
-
-@Category(IntegrationTest.class)
-public class TestTaskStatusUpdate extends QueryTestCaseBase {
-
- public TestTaskStatusUpdate() {
- super(TajoConstants.DEFAULT_DATABASE_NAME);
- }
-
- @BeforeClass
- public static void setUp() throws Exception {
- conf.set(TajoConf.ConfVars.$TEST_BROADCAST_JOIN_ENABLED.varname, "false");
- }
-
- @Test
- public final void case1() throws Exception {
- // select l_linenumber, count(1) as unique_key from lineitem group by l_linenumber;
- ResultSet res = null;
- try {
- res = executeQuery();
-
- // tpch/lineitem.tbl
- long[] expectedNumRows = new long[]{5, 2, 2, 2};
- long[] expectedNumBytes = new long[]{604, 18, 18, 8};
- long[] expectedReadBytes = new long[]{604, 604, 18, 0};
-
- assertStatus(2, expectedNumRows, expectedNumBytes, expectedReadBytes);
- } finally {
- cleanupQuery(res);
- }
- }
-
- @Test
- public final void case2() throws Exception {
- // ExternalMergeSort
- ResultSet res = null;
- try {
- res = executeQuery();
-
- // tpch/lineitem.tbl
- long[] expectedNumRows = new long[]{5, 2, 2, 2, 2, 2};
- long[] expectedNumBytes = new long[]{604, 162, 162, 138, 138, 194};
- long[] expectedReadBytes = new long[]{604, 604, 162, 0, 138, 0};
-
- assertStatus(3, expectedNumRows, expectedNumBytes, expectedReadBytes);
- } finally {
- cleanupQuery(res);
- }
- }
-
-
- @Test
- public final void case3() throws Exception {
- // Partition Scan
- ResultSet res = null;
- try {
- createColumnPartitionedTable();
-
- /*
- |-eb_1404143727281_0002_000005
- |-eb_1404143727281_0002_000004 (order by)
- |-eb_1404143727281_0002_000003 (join)
- |-eb_1404143727281_0002_000002 (scan)
- |-eb_1404143727281_0002_000001 (scan, filter)
- */
- res = executeQuery();
-
- String actualResult = resultSetToString(res);
- System.out.println(actualResult);
-
- // in/out * stage(4)
- long[] expectedNumRows = new long[]{2, 2, 5, 5, 7, 2, 2, 2};
- long[] expectedNumBytes = new long[]{8, 34, 20, 75, 109, 34, 34, 18};
- long[] expectedReadBytes = new long[]{8, 8, 20, 20, 109, 0, 34, 0};
-
- assertStatus(4, expectedNumRows, expectedNumBytes, expectedReadBytes);
- } finally {
- cleanupQuery(res);
- }
- }
-
- private void createColumnPartitionedTable() throws Exception {
- String tableName = CatalogUtil.normalizeIdentifier("ColumnPartitionedTable");
- ResultSet res = executeString(
- "create table " + tableName + " (col1 int4, col2 int4) partition by column(key float8) ");
- res.close();
-
- assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
- assertEquals(2, catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName).getSchema().size());
- assertEquals(3,
- catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName).getLogicalSchema().size());
-
- res = testBase.execute(
- "insert overwrite into " + tableName + " select l_orderkey, l_partkey, l_quantity from lineitem");
-
- res.close();
- }
-
- private void assertStatus(int numStages,
- long[] expectedNumRows,
- long[] expectedNumBytes,
- long[] expectedReadBytes) throws Exception {
- List<TajoWorker> tajoWorkers = testingCluster.getTajoWorkers();
- Collection<QueryMasterTask> finishedTasks = null;
- for (TajoWorker eachWorker: tajoWorkers) {
- finishedTasks = eachWorker.getWorkerContext().getQueryMaster().getFinishedQueryMasterTasks();
- if (finishedTasks != null && !finishedTasks.isEmpty()) {
- break;
- }
- }
-
- assertNotNull(finishedTasks);
- assertTrue(!finishedTasks.isEmpty());
-
- List<QueryMasterTask> finishedTaskList = new ArrayList<QueryMasterTask>(finishedTasks);
-
- Collections.sort(finishedTaskList, new Comparator<QueryMasterTask>() {
- @Override
- public int compare(QueryMasterTask o1, QueryMasterTask o2) {
- return o2.getQueryId().compareTo(o1.getQueryId());
- }
- });
-
- Query query = finishedTaskList.get(0).getQuery();
-
- assertNotNull(query);
-
- List<Stage> stages = new ArrayList<Stage>(query.getStages());
- assertEquals(numStages, stages.size());
-
- Collections.sort(stages, new Comparator<Stage>() {
- @Override
- public int compare(Stage o1, Stage o2) {
- return o1.getId().compareTo(o2.getId());
- }
- });
-
- int index = 0;
- for (Stage eachStage : stages) {
- TableStats inputStats = eachStage.getInputStats();
- TableStats resultStats = eachStage.getResultStats();
-
- assertNotNull(inputStats);
- assertEquals(expectedNumRows[index], inputStats.getNumRows().longValue());
- assertEquals(expectedNumBytes[index], inputStats.getNumBytes().longValue());
- assertEquals(expectedReadBytes[index], inputStats.getReadBytes().longValue());
-
- index++;
-
- assertNotNull(resultStats);
- assertEquals(expectedNumRows[index], resultStats.getNumRows().longValue());
- assertEquals(expectedNumBytes[index], resultStats.getNumBytes().longValue());
- assertEquals(expectedReadBytes[index], resultStats.getReadBytes().longValue());
-
- index++;
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestFifoScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestFifoScheduler.java b/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestFifoScheduler.java
new file mode 100644
index 0000000..e0c30a8
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/master/scheduler/TestFifoScheduler.java
@@ -0,0 +1,116 @@
+/**
+ * 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.tajo.master.scheduler;
+
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.benchmark.TPCH;
+import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.client.TajoClientImpl;
+import org.apache.tajo.client.TajoClientUtil;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.ClientProtos;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.sql.ResultSet;
+
+import static org.junit.Assert.*;
+
+public class TestFifoScheduler {
+ private static TajoTestingCluster cluster;
+ private static TajoConf conf;
+ private static TajoClient client;
+ private static String query =
+ "select l_orderkey, l_partkey from lineitem group by l_orderkey, l_partkey order by l_orderkey";
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ cluster = new TajoTestingCluster();
+ cluster.startMiniClusterInLocal(1);
+ conf = cluster.getConfiguration();
+ client = new TajoClientImpl(cluster.getConfiguration());
+ File file = TPCH.getDataFile("lineitem");
+ client.executeQueryAndGetResult("create external table default.lineitem (l_orderkey int, l_partkey int) "
+ + "using text location 'file://" + file.getAbsolutePath() + "'");
+ assertTrue(client.existTable("default.lineitem"));
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ if (client != null) client.close();
+ if (cluster != null) cluster.shutdownMiniCluster();
+ }
+
+ @Test
+ public final void testKillScheduledQuery() throws Exception {
+ ClientProtos.SubmitQueryResponse res = client.executeQuery(query);
+ ClientProtos.SubmitQueryResponse res2 = client.executeQuery(query);
+ QueryId queryId = new QueryId(res.getQueryId());
+ QueryId queryId2 = new QueryId(res2.getQueryId());
+
+ cluster.waitForQueryRunning(queryId);
+ client.killQuery(queryId2);
+ assertEquals(TajoProtos.QueryState.QUERY_KILLED, client.getQueryStatus(queryId2).getState());
+ }
+
+ @Test
+ public final void testForwardedQuery() throws Exception {
+ ClientProtos.SubmitQueryResponse res = client.executeQuery(query);
+ ClientProtos.SubmitQueryResponse res2 = client.executeQuery("select * from lineitem limit 1");
+ assertTrue(res.getIsForwarded());
+ assertFalse(res2.getIsForwarded());
+
+ QueryId queryId = new QueryId(res.getQueryId());
+ QueryId queryId2 = new QueryId(res2.getQueryId());
+ cluster.waitForQueryRunning(queryId);
+
+ assertEquals(TajoProtos.QueryState.QUERY_SUCCEEDED, client.getQueryStatus(queryId2).getState());
+ ResultSet resSet = TajoClientUtil.createResultSet(conf, client, res2);
+ assertNotNull(resSet);
+ }
+
+ @Test
+ public final void testScheduledQuery() throws Exception {
+ ClientProtos.SubmitQueryResponse res = client.executeQuery("select sleep(1) from lineitem");
+ ClientProtos.SubmitQueryResponse res2 = client.executeQuery(query);
+ ClientProtos.SubmitQueryResponse res3 = client.executeQuery(query);
+ ClientProtos.SubmitQueryResponse res4 = client.executeQuery(query);
+
+ QueryId queryId = new QueryId(res.getQueryId());
+ QueryId queryId2 = new QueryId(res2.getQueryId());
+ QueryId queryId3 = new QueryId(res3.getQueryId());
+ QueryId queryId4 = new QueryId(res4.getQueryId());
+
+ cluster.waitForQueryRunning(queryId);
+
+ assertTrue(TajoClientUtil.isQueryRunning(client.getQueryStatus(queryId).getState()));
+
+ assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId2).getState());
+ assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId3).getState());
+ assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId4).getState());
+
+ client.killQuery(queryId4);
+ client.killQuery(queryId3);
+ client.killQuery(queryId2);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/querymaster/TestIntermediateEntry.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestIntermediateEntry.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestIntermediateEntry.java
new file mode 100644
index 0000000..237fb32
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestIntermediateEntry.java
@@ -0,0 +1,53 @@
+/**
+ * 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.tajo.querymaster;
+
+import org.apache.tajo.util.Pair;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestIntermediateEntry {
+ @Test
+ public void testPage() {
+ Task.IntermediateEntry interm = new Task.IntermediateEntry(-1, -1, 1, null);
+
+ List<Pair<Long, Integer>> pages = new ArrayList<Pair<Long, Integer>>();
+ pages.add(new Pair(0L, 1441275));
+ pages.add(new Pair(1441275L, 1447446));
+ pages.add(new Pair(2888721L, 1442507));
+
+ interm.setPages(pages);
+
+ long splitBytes = 3 * 1024 * 1024;
+
+ List<Pair<Long, Long>> splits = interm.split(splitBytes, splitBytes);
+ assertEquals(2, splits.size());
+
+ long[][] expected = { {0, 1441275 + 1447446}, {1441275 + 1447446, 1442507} };
+ for (int i = 0; i < 2; i++) {
+ Pair<Long, Long> eachSplit = splits.get(i);
+ assertEquals(expected[i][0], eachSplit.getFirst().longValue());
+ assertEquals(expected[i][1], eachSplit.getSecond().longValue());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
new file mode 100644
index 0000000..a125196
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestKillQuery.java
@@ -0,0 +1,125 @@
+/**
+ * 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.tajo.querymaster;
+
+import org.apache.tajo.*;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.benchmark.TPCH;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.client.TajoClientImpl;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.global.GlobalPlanner;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.master.event.QueryEvent;
+import org.apache.tajo.master.event.QueryEventType;
+import org.apache.tajo.session.Session;
+import org.apache.tajo.plan.LogicalOptimizer;
+import org.apache.tajo.plan.LogicalPlan;
+import org.apache.tajo.plan.LogicalPlanner;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.junit.Assert.*;
+
+public class TestKillQuery {
+ private static TajoTestingCluster cluster;
+ private static TajoConf conf;
+ private static TajoClient client;
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ cluster = new TajoTestingCluster();
+ cluster.startMiniClusterInLocal(1);
+ conf = cluster.getConfiguration();
+ client = new TajoClientImpl(cluster.getConfiguration());
+ File file = TPCH.getDataFile("lineitem");
+ client.executeQueryAndGetResult("create external table default.lineitem (l_orderkey int, l_partkey int) "
+ + "using text location 'file://" + file.getAbsolutePath() + "'");
+ assertTrue(client.existTable("default.lineitem"));
+ }
+
+ @AfterClass
+ public static void tearDown() throws IOException {
+ if (client != null) client.close();
+ if (cluster != null) cluster.shutdownMiniCluster();
+ }
+
+ @Test
+ public final void testKillQueryFromInitState() throws Exception {
+ SQLAnalyzer analyzer = new SQLAnalyzer();
+ QueryContext defaultContext = LocalTajoTestingUtility.createDummyContext(conf);
+ Session session = LocalTajoTestingUtility.createDummySession();
+ CatalogService catalog = cluster.getMaster().getCatalog();
+ String query = "select l_orderkey, l_partkey from lineitem group by l_orderkey, l_partkey order by l_orderkey";
+
+ LogicalPlanner planner = new LogicalPlanner(catalog);
+ LogicalOptimizer optimizer = new LogicalOptimizer(conf);
+ Expr expr = analyzer.parse(query);
+ LogicalPlan plan = planner.createPlan(defaultContext, expr);
+
+ optimizer.optimize(plan);
+
+ QueryId queryId = QueryIdFactory.newQueryId(System.currentTimeMillis(), 0);
+ QueryContext queryContext = new QueryContext(conf);
+ MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
+ GlobalPlanner globalPlanner = new GlobalPlanner(conf, catalog);
+ globalPlanner.build(masterPlan);
+
+ QueryMaster qm = cluster.getTajoWorkers().get(0).getWorkerContext().getQueryMaster();
+ QueryMasterTask queryMasterTask = new QueryMasterTask(qm.getContext(),
+ queryId, session, defaultContext, expr.toJson(), plan.getRootBlock().getRoot().toJson());
+
+ queryMasterTask.init(conf);
+ queryMasterTask.getQueryTaskContext().getDispatcher().start();
+ queryMasterTask.startQuery();
+
+ try{
+ cluster.waitForQueryState(queryMasterTask.getQuery(), TajoProtos.QueryState.QUERY_RUNNING, 2);
+ } finally {
+ assertEquals(TajoProtos.QueryState.QUERY_RUNNING, queryMasterTask.getQuery().getSynchronizedState());
+ }
+
+ Stage stage = queryMasterTask.getQuery().getStages().iterator().next();
+ assertNotNull(stage);
+
+ try{
+ cluster.waitForStageState(stage, StageState.INITED, 2);
+ } finally {
+ assertEquals(StageState.INITED, stage.getSynchronizedState());
+ }
+
+ // fire kill event
+ Query q = queryMasterTask.getQuery();
+ q.handle(new QueryEvent(queryId, QueryEventType.KILL));
+
+ try{
+ cluster.waitForQueryState(queryMasterTask.getQuery(), TajoProtos.QueryState.QUERY_KILLED, 50);
+ } finally {
+ assertEquals(TajoProtos.QueryState.QUERY_KILLED, queryMasterTask.getQuery().getSynchronizedState());
+ }
+ queryMasterTask.stop();
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/querymaster/TestQueryProgress.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestQueryProgress.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestQueryProgress.java
new file mode 100644
index 0000000..7c61670
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestQueryProgress.java
@@ -0,0 +1,75 @@
+/**
+ * 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.tajo.querymaster;
+
+import org.apache.tajo.*;
+import org.apache.tajo.client.QueryStatus;
+import org.apache.tajo.client.TajoClient;
+import org.apache.tajo.client.TajoClientImpl;
+import org.apache.tajo.client.TajoClientUtil;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.ipc.ClientProtos;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.*;
+
+@Category(IntegrationTest.class)
+public class TestQueryProgress {
+ private static TajoTestingCluster cluster;
+ private static TajoConf conf;
+ private static TajoClient client;
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ cluster = TpchTestBase.getInstance().getTestingCluster();
+ conf = cluster.getConfiguration();
+ client = new TajoClientImpl(conf);
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ client.close();
+ }
+
+ @Test(timeout = 10000)
+ public final void testQueryProgress() throws Exception {
+ ClientProtos.SubmitQueryResponse res = client.executeQuery("select l_orderkey from lineitem group by l_orderkey");
+ QueryId queryId = new QueryId(res.getQueryId());
+
+ float prevProgress = 0;
+ while (true) {
+ QueryStatus status = client.getQueryStatus(queryId);
+ if (status == null) continue;
+
+ float progress = status.getProgress();
+
+ if (prevProgress > progress) {
+ fail("Previous progress: " + prevProgress + ", Current progress : " + progress);
+ }
+ prevProgress = progress;
+ assertTrue(progress <= 1.0f);
+
+ if (TajoClientUtil.isQueryComplete(status.getState())) break;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/querymaster/TestTaskStatusUpdate.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/querymaster/TestTaskStatusUpdate.java b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestTaskStatusUpdate.java
new file mode 100644
index 0000000..ab5375c
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/querymaster/TestTaskStatusUpdate.java
@@ -0,0 +1,194 @@
+/**
+ * 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.tajo.querymaster;
+
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.QueryTestCaseBase;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.worker.TajoWorker;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.sql.ResultSet;
+import java.util.*;
+
+import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME;
+import static org.junit.Assert.*;
+
+@Category(IntegrationTest.class)
+public class TestTaskStatusUpdate extends QueryTestCaseBase {
+
+ public TestTaskStatusUpdate() {
+ super(TajoConstants.DEFAULT_DATABASE_NAME);
+ }
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ conf.set(TajoConf.ConfVars.$TEST_BROADCAST_JOIN_ENABLED.varname, "false");
+ }
+
+ @Test
+ public final void case1() throws Exception {
+ // select l_linenumber, count(1) as unique_key from lineitem group by l_linenumber;
+ ResultSet res = null;
+ try {
+ res = executeQuery();
+
+ // tpch/lineitem.tbl
+ long[] expectedNumRows = new long[]{5, 2, 2, 2};
+ long[] expectedNumBytes = new long[]{604, 18, 18, 8};
+ long[] expectedReadBytes = new long[]{604, 604, 18, 0};
+
+ assertStatus(2, expectedNumRows, expectedNumBytes, expectedReadBytes);
+ } finally {
+ cleanupQuery(res);
+ }
+ }
+
+ @Test
+ public final void case2() throws Exception {
+ // ExternalMergeSort
+ ResultSet res = null;
+ try {
+ res = executeQuery();
+
+ // tpch/lineitem.tbl
+ long[] expectedNumRows = new long[]{5, 2, 2, 2, 2, 2};
+ long[] expectedNumBytes = new long[]{604, 162, 162, 138, 138, 194};
+ long[] expectedReadBytes = new long[]{604, 604, 162, 0, 138, 0};
+
+ assertStatus(3, expectedNumRows, expectedNumBytes, expectedReadBytes);
+ } finally {
+ cleanupQuery(res);
+ }
+ }
+
+
+ @Test
+ public final void case3() throws Exception {
+ // Partition Scan
+ ResultSet res = null;
+ try {
+ createColumnPartitionedTable();
+
+ /*
+ |-eb_1404143727281_0002_000005
+ |-eb_1404143727281_0002_000004 (order by)
+ |-eb_1404143727281_0002_000003 (join)
+ |-eb_1404143727281_0002_000002 (scan)
+ |-eb_1404143727281_0002_000001 (scan, filter)
+ */
+ res = executeQuery();
+
+ String actualResult = resultSetToString(res);
+ System.out.println(actualResult);
+
+ // in/out * stage(4)
+ long[] expectedNumRows = new long[]{2, 2, 5, 5, 7, 2, 2, 2};
+ long[] expectedNumBytes = new long[]{8, 34, 20, 75, 109, 34, 34, 18};
+ long[] expectedReadBytes = new long[]{8, 8, 20, 20, 109, 0, 34, 0};
+
+ assertStatus(4, expectedNumRows, expectedNumBytes, expectedReadBytes);
+ } finally {
+ cleanupQuery(res);
+ }
+ }
+
+ private void createColumnPartitionedTable() throws Exception {
+ String tableName = CatalogUtil.normalizeIdentifier("ColumnPartitionedTable");
+ ResultSet res = executeString(
+ "create table " + tableName + " (col1 int4, col2 int4) partition by column(key float8) ");
+ res.close();
+
+ assertTrue(catalog.existsTable(DEFAULT_DATABASE_NAME, tableName));
+ assertEquals(2, catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName).getSchema().size());
+ assertEquals(3,
+ catalog.getTableDesc(DEFAULT_DATABASE_NAME, tableName).getLogicalSchema().size());
+
+ res = testBase.execute(
+ "insert overwrite into " + tableName + " select l_orderkey, l_partkey, l_quantity from lineitem");
+
+ res.close();
+ }
+
+ private void assertStatus(int numStages,
+ long[] expectedNumRows,
+ long[] expectedNumBytes,
+ long[] expectedReadBytes) throws Exception {
+ List<TajoWorker> tajoWorkers = testingCluster.getTajoWorkers();
+ Collection<QueryMasterTask> finishedTasks = null;
+ for (TajoWorker eachWorker: tajoWorkers) {
+ finishedTasks = eachWorker.getWorkerContext().getQueryMaster().getFinishedQueryMasterTasks();
+ if (finishedTasks != null && !finishedTasks.isEmpty()) {
+ break;
+ }
+ }
+
+ assertNotNull(finishedTasks);
+ assertTrue(!finishedTasks.isEmpty());
+
+ List<QueryMasterTask> finishedTaskList = new ArrayList<QueryMasterTask>(finishedTasks);
+
+ Collections.sort(finishedTaskList, new Comparator<QueryMasterTask>() {
+ @Override
+ public int compare(QueryMasterTask o1, QueryMasterTask o2) {
+ return o2.getQueryId().compareTo(o1.getQueryId());
+ }
+ });
+
+ Query query = finishedTaskList.get(0).getQuery();
+
+ assertNotNull(query);
+
+ List<Stage> stages = new ArrayList<Stage>(query.getStages());
+ assertEquals(numStages, stages.size());
+
+ Collections.sort(stages, new Comparator<Stage>() {
+ @Override
+ public int compare(Stage o1, Stage o2) {
+ return o1.getId().compareTo(o2.getId());
+ }
+ });
+
+ int index = 0;
+ for (Stage eachStage : stages) {
+ TableStats inputStats = eachStage.getInputStats();
+ TableStats resultStats = eachStage.getResultStats();
+
+ assertNotNull(inputStats);
+ assertEquals(expectedNumRows[index], inputStats.getNumRows().longValue());
+ assertEquals(expectedNumBytes[index], inputStats.getNumBytes().longValue());
+ assertEquals(expectedReadBytes[index], inputStats.getReadBytes().longValue());
+
+ index++;
+
+ assertNotNull(resultStats);
+ assertEquals(expectedNumRows[index], resultStats.getNumRows().longValue());
+ assertEquals(expectedNumBytes[index], resultStats.getNumBytes().longValue());
+ assertEquals(expectedReadBytes[index], resultStats.getReadBytes().longValue());
+
+ index++;
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/scheduler/TestFifoScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/scheduler/TestFifoScheduler.java b/tajo-core/src/test/java/org/apache/tajo/scheduler/TestFifoScheduler.java
deleted file mode 100644
index acd6b71..0000000
--- a/tajo-core/src/test/java/org/apache/tajo/scheduler/TestFifoScheduler.java
+++ /dev/null
@@ -1,116 +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.tajo.scheduler;
-
-import org.apache.tajo.QueryId;
-import org.apache.tajo.TajoProtos;
-import org.apache.tajo.TajoTestingCluster;
-import org.apache.tajo.benchmark.TPCH;
-import org.apache.tajo.client.TajoClient;
-import org.apache.tajo.client.TajoClientImpl;
-import org.apache.tajo.client.TajoClientUtil;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.ipc.ClientProtos;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.File;
-import java.sql.ResultSet;
-
-import static org.junit.Assert.*;
-
-public class TestFifoScheduler {
- private static TajoTestingCluster cluster;
- private static TajoConf conf;
- private static TajoClient client;
- private static String query =
- "select l_orderkey, l_partkey from lineitem group by l_orderkey, l_partkey order by l_orderkey";
-
- @BeforeClass
- public static void setUp() throws Exception {
- cluster = new TajoTestingCluster();
- cluster.startMiniClusterInLocal(1);
- conf = cluster.getConfiguration();
- client = new TajoClientImpl(cluster.getConfiguration());
- File file = TPCH.getDataFile("lineitem");
- client.executeQueryAndGetResult("create external table default.lineitem (l_orderkey int, l_partkey int) "
- + "using text location 'file://" + file.getAbsolutePath() + "'");
- assertTrue(client.existTable("default.lineitem"));
- }
-
- @AfterClass
- public static void tearDown() throws Exception {
- if (client != null) client.close();
- if (cluster != null) cluster.shutdownMiniCluster();
- }
-
- @Test
- public final void testKillScheduledQuery() throws Exception {
- ClientProtos.SubmitQueryResponse res = client.executeQuery(query);
- ClientProtos.SubmitQueryResponse res2 = client.executeQuery(query);
- QueryId queryId = new QueryId(res.getQueryId());
- QueryId queryId2 = new QueryId(res2.getQueryId());
-
- cluster.waitForQueryRunning(queryId);
- client.killQuery(queryId2);
- assertEquals(TajoProtos.QueryState.QUERY_KILLED, client.getQueryStatus(queryId2).getState());
- }
-
- @Test
- public final void testForwardedQuery() throws Exception {
- ClientProtos.SubmitQueryResponse res = client.executeQuery(query);
- ClientProtos.SubmitQueryResponse res2 = client.executeQuery("select * from lineitem limit 1");
- assertTrue(res.getIsForwarded());
- assertFalse(res2.getIsForwarded());
-
- QueryId queryId = new QueryId(res.getQueryId());
- QueryId queryId2 = new QueryId(res2.getQueryId());
- cluster.waitForQueryRunning(queryId);
-
- assertEquals(TajoProtos.QueryState.QUERY_SUCCEEDED, client.getQueryStatus(queryId2).getState());
- ResultSet resSet = TajoClientUtil.createResultSet(conf, client, res2);
- assertNotNull(resSet);
- }
-
- @Test
- public final void testScheduledQuery() throws Exception {
- ClientProtos.SubmitQueryResponse res = client.executeQuery("select sleep(1) from lineitem");
- ClientProtos.SubmitQueryResponse res2 = client.executeQuery(query);
- ClientProtos.SubmitQueryResponse res3 = client.executeQuery(query);
- ClientProtos.SubmitQueryResponse res4 = client.executeQuery(query);
-
- QueryId queryId = new QueryId(res.getQueryId());
- QueryId queryId2 = new QueryId(res2.getQueryId());
- QueryId queryId3 = new QueryId(res3.getQueryId());
- QueryId queryId4 = new QueryId(res4.getQueryId());
-
- cluster.waitForQueryRunning(queryId);
-
- assertTrue(TajoClientUtil.isQueryRunning(client.getQueryStatus(queryId).getState()));
-
- assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId2).getState());
- assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId3).getState());
- assertEquals(TajoProtos.QueryState.QUERY_MASTER_INIT, client.getQueryStatus(queryId4).getState());
-
- client.killQuery(queryId4);
- client.killQuery(queryId3);
- client.killQuery(queryId2);
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/util/TestJSPUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/util/TestJSPUtil.java b/tajo-core/src/test/java/org/apache/tajo/util/TestJSPUtil.java
index 2e00138..f3b6936 100644
--- a/tajo-core/src/test/java/org/apache/tajo/util/TestJSPUtil.java
+++ b/tajo-core/src/test/java/org/apache/tajo/util/TestJSPUtil.java
@@ -23,7 +23,7 @@ import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.TaskId;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent;
-import org.apache.tajo.master.querymaster.Task;
+import org.apache.tajo.querymaster.Task;
import org.junit.Test;
import java.util.ArrayList;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java b/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java
index 632e9c2..45282aa 100644
--- a/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java
+++ b/tajo-core/src/test/java/org/apache/tajo/util/history/TestHistoryWriterReader.java
@@ -28,7 +28,7 @@ import org.apache.tajo.TajoProtos.TaskAttemptState;
import org.apache.tajo.catalog.proto.CatalogProtos.TableStatsProto;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.master.querymaster.QueryInfo;
+import org.apache.tajo.master.QueryInfo;
import org.apache.tajo.util.TajoIdUtils;
import org.junit.After;
import org.junit.Before;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/worker/TestHistory.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/worker/TestHistory.java b/tajo-core/src/test/java/org/apache/tajo/worker/TestHistory.java
index 77aa1d4..9a51422 100644
--- a/tajo-core/src/test/java/org/apache/tajo/worker/TestHistory.java
+++ b/tajo-core/src/test/java/org/apache/tajo/worker/TestHistory.java
@@ -28,7 +28,7 @@ import org.apache.tajo.client.TajoClient;
import org.apache.tajo.client.TajoClientImpl;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.master.TajoMaster;
-import org.apache.tajo.master.querymaster.QueryInfo;
+import org.apache.tajo.master.QueryInfo;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-dist/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-dist/pom.xml b/tajo-dist/pom.xml
index e6b4479..d469ba9 100644
--- a/tajo-dist/pom.xml
+++ b/tajo-dist/pom.xml
@@ -223,10 +223,10 @@
<mainClass>org.apache.hadoop.yarn.state.VisualizeStateMachine</mainClass>
<arguments>
<argument>Tajo</argument>
- <argument>org.apache.tajo.master.querymaster.Query,
- org.apache.tajo.master.querymaster.Stage,
- org.apache.tajo.master.querymaster.Task,
- org.apache.tajo.master.querymaster.TaskAttempt
+ <argument>org.apache.tajo.querymaster.Query,
+ org.apache.tajo.querymaster.Stage,
+ org.apache.tajo.querymaster.Task,
+ org.apache.tajo.querymaster.TaskAttempt
</argument>
<argument>Tajo.gv</argument>
</arguments>
[16/16] tajo git commit: Merge branch 'master' of
https://git-wip-us.apache.org/repos/asf/tajo into index_support
Posted by ji...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/tajo into index_support
Conflicts:
tajo-core/src/main/java/org/apache/tajo/querymaster/QueryMasterTask.java
Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/e04c65fd
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/e04c65fd
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/e04c65fd
Branch: refs/heads/index_support
Commit: e04c65fdde7587322e2b6c47b7b9ffe951d9e8bc
Parents: 071c5d0 1c29c1c
Author: Jihoon Son <ji...@apache.org>
Authored: Fri Jan 9 01:17:02 2015 +0900
Committer: Jihoon Son <ji...@apache.org>
Committed: Fri Jan 9 01:17:02 2015 +0900
----------------------------------------------------------------------
CHANGES | 4 +
.../tajo/engine/function/builtin/AvgDouble.java | 25 +-
.../tajo/engine/function/builtin/AvgFloat.java | 12 +-
.../tajo/engine/function/builtin/AvgInt.java | 14 +-
.../tajo/engine/function/builtin/AvgLong.java | 27 +-
.../tajo/engine/function/builtin/Max.java | 76 +
.../tajo/engine/function/builtin/MaxDouble.java | 43 +-
.../tajo/engine/function/builtin/MaxFloat.java | 43 +-
.../tajo/engine/function/builtin/MaxInt.java | 44 +-
.../tajo/engine/function/builtin/MaxLong.java | 32 +-
.../tajo/engine/function/builtin/MaxString.java | 52 +-
.../tajo/engine/function/builtin/Min.java | 76 +
.../tajo/engine/function/builtin/MinDouble.java | 41 +-
.../tajo/engine/function/builtin/MinFloat.java | 42 +-
.../tajo/engine/function/builtin/MinInt.java | 43 +-
.../tajo/engine/function/builtin/MinLong.java | 32 +-
.../tajo/engine/function/builtin/MinString.java | 46 +-
.../tajo/engine/function/builtin/SumDouble.java | 34 +-
.../tajo/engine/function/builtin/SumFloat.java | 33 +-
.../tajo/engine/function/builtin/SumInt.java | 32 +-
.../tajo/engine/function/builtin/SumLong.java | 34 +-
.../DistinctGroupbySortAggregationExec.java | 41 +-
.../apache/tajo/engine/query/QueryContext.java | 2 +-
.../main/java/org/apache/tajo/ha/HAService.java | 56 +
.../org/apache/tajo/ha/HAServiceHDFSImpl.java | 316 +++++
.../java/org/apache/tajo/ha/TajoMasterInfo.java | 89 ++
.../tajo/master/AbstractTaskScheduler.java | 56 -
.../org/apache/tajo/master/ContainerProxy.java | 2 +-
.../tajo/master/DefaultTaskScheduler.java | 928 ------------
.../apache/tajo/master/FetchScheduleEvent.java | 40 -
.../org/apache/tajo/master/FragmentPair.java | 73 -
.../org/apache/tajo/master/GlobalEngine.java | 2 +-
.../NonForwardQueryResultFileScanner.java | 164 ---
.../master/NonForwardQueryResultScanner.java | 46 -
.../NonForwardQueryResultSystemScanner.java | 600 --------
.../java/org/apache/tajo/master/QueryInfo.java | 235 +++
.../org/apache/tajo/master/QueryJobManager.java | 311 ++++
.../apache/tajo/master/ScheduledFetches.java | 49 -
.../apache/tajo/master/TajoContainerProxy.java | 2 +-
.../java/org/apache/tajo/master/TajoMaster.java | 11 +-
.../tajo/master/TajoMasterClientService.java | 13 +-
.../apache/tajo/master/TajoMasterService.java | 2 -
.../tajo/master/TaskSchedulerContext.java | 65 -
.../tajo/master/TaskSchedulerFactory.java | 69 -
.../tajo/master/event/QueryCompletedEvent.java | 2 +-
.../tajo/master/event/QueryStartEvent.java | 2 +-
.../tajo/master/event/StageCompletedEvent.java | 2 +-
.../event/TaskAttemptToSchedulerEvent.java | 2 +-
.../apache/tajo/master/exec/DDLExecutor.java | 1 -
.../exec/NonForwardQueryResultFileScanner.java | 164 +++
.../exec/NonForwardQueryResultScanner.java | 46 +
.../NonForwardQueryResultSystemScanner.java | 600 ++++++++
.../apache/tajo/master/exec/QueryExecutor.java | 9 +-
.../org/apache/tajo/master/ha/HAService.java | 56 -
.../tajo/master/ha/HAServiceHDFSImpl.java | 318 -----
.../apache/tajo/master/ha/TajoMasterInfo.java | 89 --
.../master/metrics/CatalogMetricsGaugeSet.java | 56 -
.../metrics/WorkerResourceMetricsGaugeSet.java | 74 -
.../apache/tajo/master/querymaster/Query.java | 783 ----------
.../master/querymaster/QueryInProgress.java | 300 ----
.../tajo/master/querymaster/QueryInfo.java | 235 ---
.../tajo/master/querymaster/QueryJobEvent.java | 45 -
.../master/querymaster/QueryJobManager.java | 310 ----
.../tajo/master/querymaster/QueryMaster.java | 631 --------
.../querymaster/QueryMasterManagerService.java | 263 ----
.../master/querymaster/QueryMasterRunner.java | 149 --
.../master/querymaster/QueryMasterTask.java | 647 ---------
.../tajo/master/querymaster/Repartitioner.java | 1251 ----------------
.../apache/tajo/master/querymaster/Stage.java | 1342 ------------------
.../tajo/master/querymaster/StageState.java | 30 -
.../apache/tajo/master/querymaster/Task.java | 907 ------------
.../tajo/master/querymaster/TaskAttempt.java | 443 ------
.../master/rm/TajoWorkerResourceManager.java | 3 +-
.../tajo/master/rm/WorkerResourceManager.java | 2 +-
.../master/scheduler/QuerySchedulingInfo.java | 55 +
.../apache/tajo/master/scheduler/Scheduler.java | 41 +
.../master/scheduler/SchedulingAlgorithms.java | 47 +
.../master/scheduler/SimpleFifoScheduler.java | 147 ++
.../master/session/InvalidSessionException.java | 25 -
.../session/NoSuchSessionVariableException.java | 25 -
.../org/apache/tajo/master/session/Session.java | 196 ---
.../tajo/master/session/SessionConstants.java | 23 -
.../tajo/master/session/SessionEvent.java | 34 -
.../tajo/master/session/SessionEventType.java | 24 -
.../session/SessionLivelinessMonitor.java | 53 -
.../tajo/master/session/SessionManager.java | 144 --
.../tajo/metrics/CatalogMetricsGaugeSet.java | 56 +
.../metrics/WorkerResourceMetricsGaugeSet.java | 74 +
.../tajo/querymaster/AbstractTaskScheduler.java | 56 +
.../tajo/querymaster/DefaultTaskScheduler.java | 926 ++++++++++++
.../tajo/querymaster/FetchScheduleEvent.java | 40 +
.../java/org/apache/tajo/querymaster/Query.java | 783 ++++++++++
.../tajo/querymaster/QueryInProgress.java | 301 ++++
.../apache/tajo/querymaster/QueryJobEvent.java | 46 +
.../apache/tajo/querymaster/QueryMaster.java | 631 ++++++++
.../querymaster/QueryMasterManagerService.java | 262 ++++
.../tajo/querymaster/QueryMasterTask.java | 650 +++++++++
.../apache/tajo/querymaster/Repartitioner.java | 1250 ++++++++++++++++
.../java/org/apache/tajo/querymaster/Stage.java | 1342 ++++++++++++++++++
.../org/apache/tajo/querymaster/StageState.java | 30 +
.../java/org/apache/tajo/querymaster/Task.java | 897 ++++++++++++
.../apache/tajo/querymaster/TaskAttempt.java | 443 ++++++
.../tajo/querymaster/TaskSchedulerContext.java | 65 +
.../tajo/querymaster/TaskSchedulerFactory.java | 68 +
.../tajo/scheduler/QuerySchedulingInfo.java | 55 -
.../org/apache/tajo/scheduler/Scheduler.java | 41 -
.../tajo/scheduler/SchedulingAlgorithms.java | 47 -
.../tajo/scheduler/SimpleFifoScheduler.java | 147 --
.../tajo/session/InvalidSessionException.java | 25 +
.../session/NoSuchSessionVariableException.java | 25 +
.../java/org/apache/tajo/session/Session.java | 196 +++
.../apache/tajo/session/SessionConstants.java | 23 +
.../org/apache/tajo/session/SessionEvent.java | 34 +
.../apache/tajo/session/SessionEventType.java | 24 +
.../tajo/session/SessionLivelinessMonitor.java | 53 +
.../org/apache/tajo/session/SessionManager.java | 144 ++
.../main/java/org/apache/tajo/util/JSPUtil.java | 10 +-
.../apache/tajo/util/history/HistoryReader.java | 2 +-
.../apache/tajo/util/history/HistoryWriter.java | 2 +-
.../java/org/apache/tajo/worker/FetchImpl.java | 4 +-
.../tajo/worker/TajoResourceAllocator.java | 6 +-
.../java/org/apache/tajo/worker/TajoWorker.java | 6 +-
.../tajo/worker/TajoWorkerClientService.java | 2 +-
tajo-core/src/main/resources/tajo-default.xml | 2 +-
.../resources/webapps/admin/catalogview.jsp | 2 +-
.../main/resources/webapps/admin/cluster.jsp | 4 +-
.../src/main/resources/webapps/admin/index.jsp | 6 +-
.../src/main/resources/webapps/admin/query.jsp | 4 +-
.../resources/webapps/admin/query_executor.jsp | 2 +-
.../src/main/resources/webapps/worker/index.jsp | 4 +-
.../resources/webapps/worker/querydetail.jsp | 4 +-
.../main/resources/webapps/worker/queryplan.jsp | 6 +-
.../resources/webapps/worker/querytasks.jsp | 2 +-
.../src/main/resources/webapps/worker/task.jsp | 8 +-
.../apache/tajo/LocalTajoTestingUtility.java | 2 +-
.../org/apache/tajo/TajoTestingCluster.java | 8 +-
.../engine/function/TestBuiltinFunctions.java | 234 +++
.../tajo/engine/planner/TestLogicalPlanner.java | 2 +-
.../planner/physical/TestPhysicalPlanner.java | 2 +-
.../tajo/engine/query/TestGroupByQuery.java | 12 +-
.../tajo/engine/query/TestJoinBroadcast.java | 2 +-
.../tajo/engine/query/TestTablePartitions.java | 2 +-
.../apache/tajo/ha/TestHAServiceHDFSImpl.java | 153 ++
.../TestNonForwardQueryResultSystemScanner.java | 4 +-
.../apache/tajo/master/TestRepartitioner.java | 8 +-
.../tajo/master/ha/TestHAServiceHDFSImpl.java | 158 ---
.../querymaster/TestIntermediateEntry.java | 53 -
.../tajo/master/querymaster/TestKillQuery.java | 125 --
.../master/querymaster/TestQueryProgress.java | 75 -
.../querymaster/TestTaskStatusUpdate.java | 194 ---
.../master/scheduler/TestFifoScheduler.java | 116 ++
.../tajo/querymaster/TestIntermediateEntry.java | 53 +
.../apache/tajo/querymaster/TestKillQuery.java | 125 ++
.../tajo/querymaster/TestQueryProgress.java | 75 +
.../tajo/querymaster/TestTaskStatusUpdate.java | 194 +++
.../tajo/scheduler/TestFifoScheduler.java | 116 --
.../java/org/apache/tajo/util/TestJSPUtil.java | 2 +-
.../util/history/TestHistoryWriterReader.java | 2 +-
.../org/apache/tajo/worker/TestHistory.java | 2 +-
.../testAvgLongOverflow.sql | 1 +
.../testAvgLongOverflow.result | 3 +
.../testGroupByWithNullData2.result | 2 +-
.../testGroupByWithNullData3.result | 2 +-
.../testGroupByWithNullData4.result | 2 +-
.../testGroupByWithNullData6.result | 2 +-
.../testGroupByWithNullData7.result | 2 +-
.../testGroupByWithNullData8.result | 2 +-
.../testLeftOuterJoinWithEmptyTable2.result | 10 +-
.../testLeftOuterJoinWithEmptyTable4.result | 2 +-
.../testLeftOuterJoinWithEmptyTable2.result | 10 +-
.../testLeftOuterJoinWithEmptyTable4.result | 2 +-
.../testLeftOuterJoinWithEmptyTable5.result | 4 +-
tajo-dist/pom.xml | 8 +-
173 files changed, 11976 insertions(+), 12138 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/e04c65fd/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/e04c65fd/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/e04c65fd/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/e04c65fd/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java
index 0000000,f645dc5..00a5362
mode 000000,100644..100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultSystemScanner.java
@@@ -1,0 -1,616 +1,600 @@@
+ /**
+ * 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.tajo.master.exec;
+
+ import java.io.IOException;
+ import java.util.ArrayList;
+ import java.util.List;
+ import java.util.Stack;
+
+ import org.apache.commons.logging.Log;
+ import org.apache.commons.logging.LogFactory;
+ import org.apache.tajo.QueryId;
+ import org.apache.tajo.TaskAttemptId;
+ import org.apache.tajo.TaskId;
+ import org.apache.tajo.catalog.CatalogUtil;
+ import org.apache.tajo.catalog.Column;
+ import org.apache.tajo.catalog.Schema;
+ import org.apache.tajo.catalog.TableDesc;
+ import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.DatabaseProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.IndexProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableDescriptorProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableOptionProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TablePartitionProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableStatsProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TablespaceProto;
++import org.apache.tajo.catalog.proto.CatalogProtos.*;
+ import org.apache.tajo.catalog.statistics.TableStats;
+ import org.apache.tajo.common.TajoDataTypes.DataType;
+ import org.apache.tajo.conf.TajoConf;
+ import org.apache.tajo.datum.DatumFactory;
+ import org.apache.tajo.engine.codegen.CompilationError;
+ import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
+ import org.apache.tajo.engine.planner.Projector;
+ import org.apache.tajo.engine.planner.global.ExecutionBlock;
+ import org.apache.tajo.engine.planner.global.ExecutionBlockCursor;
+ import org.apache.tajo.engine.planner.global.GlobalPlanner;
+ import org.apache.tajo.engine.planner.global.MasterPlan;
+ import org.apache.tajo.engine.planner.physical.PhysicalExec;
+ import org.apache.tajo.engine.query.QueryContext;
+ import org.apache.tajo.master.TajoMaster.MasterContext;
+ import org.apache.tajo.plan.LogicalPlan;
+ import org.apache.tajo.plan.PlanningException;
+ import org.apache.tajo.plan.expr.EvalNode;
+ import org.apache.tajo.plan.logical.IndexScanNode;
+ import org.apache.tajo.plan.logical.LogicalNode;
+ import org.apache.tajo.plan.logical.ScanNode;
+ import org.apache.tajo.storage.RowStoreUtil;
+ import org.apache.tajo.storage.Tuple;
+ import org.apache.tajo.storage.VTuple;
+ import org.apache.tajo.storage.RowStoreUtil.RowStoreEncoder;
+ import org.apache.tajo.util.KeyValueSet;
+ import org.apache.tajo.util.TUtil;
+ import org.apache.tajo.worker.TaskAttemptContext;
+
+ import com.google.protobuf.ByteString;
+
+ public class NonForwardQueryResultSystemScanner implements NonForwardQueryResultScanner {
+
+ private final Log LOG = LogFactory.getLog(getClass());
+
+ private MasterContext masterContext;
+ private LogicalPlan logicalPlan;
+ private final QueryId queryId;
+ private final String sessionId;
+ private TaskAttemptContext taskContext;
+ private int currentRow;
+ private long maxRow;
+ private TableDesc tableDesc;
+ private Schema outSchema;
+ private RowStoreEncoder encoder;
+ private PhysicalExec physicalExec;
+
+ public NonForwardQueryResultSystemScanner(MasterContext context, LogicalPlan plan, QueryId queryId,
+ String sessionId, int maxRow) {
+ masterContext = context;
+ logicalPlan = plan;
+ this.queryId = queryId;
+ this.sessionId = sessionId;
+ this.maxRow = maxRow;
+
+ }
+
+ @Override
+ public void init() throws IOException {
+ QueryContext queryContext = new QueryContext(masterContext.getConf());
+ currentRow = 0;
+
+ MasterPlan masterPlan = new MasterPlan(queryId, queryContext, logicalPlan);
+ GlobalPlanner globalPlanner = new GlobalPlanner(masterContext.getConf(), masterContext.getCatalog());
+ try {
+ globalPlanner.build(masterPlan);
+ } catch (PlanningException e) {
+ throw new RuntimeException(e);
+ }
+
+ ExecutionBlockCursor cursor = new ExecutionBlockCursor(masterPlan);
+ ExecutionBlock leafBlock = null;
+ while (cursor.hasNext()) {
+ ExecutionBlock block = cursor.nextBlock();
+ if (masterPlan.isLeaf(block)) {
+ leafBlock = block;
+ break;
+ }
+ }
+
+ taskContext = new TaskAttemptContext(queryContext, null,
+ new TaskAttemptId(new TaskId(leafBlock.getId(), 0), 0),
+ null, null);
+ physicalExec = new SimplePhysicalPlannerImpl(masterContext.getConf())
+ .createPlan(taskContext, leafBlock.getPlan());
+
+ tableDesc = new TableDesc("table_"+System.currentTimeMillis(), physicalExec.getSchema(),
+ new TableMeta(StoreType.SYSTEM, new KeyValueSet()), null);
+ outSchema = physicalExec.getSchema();
+ encoder = RowStoreUtil.createEncoder(getLogicalSchema());
+
+ physicalExec.init();
+ }
+
+ @Override
+ public void close() throws Exception {
+ tableDesc = null;
+ outSchema = null;
+ encoder = null;
+ if (physicalExec != null) {
+ try {
+ physicalExec.close();
+ } catch (Exception ignored) {}
+ }
+ physicalExec = null;
+ currentRow = -1;
+ }
+
+ private List<Tuple> getTablespaces(Schema outSchema) {
+ List<TablespaceProto> tablespaces = masterContext.getCatalog().getAllTablespaces();
+ List<Tuple> tuples = new ArrayList<Tuple>(tablespaces.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+
+ for (TablespaceProto tablespace: tablespaces) {
+ aTuple = new VTuple(outSchema.size());
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column column = columns.get(fieldId);
+ if ("space_id".equalsIgnoreCase(column.getSimpleName())) {
+ if (tablespace.hasId()) {
+ aTuple.put(fieldId, DatumFactory.createInt4(tablespace.getId()));
+ } else {
+ aTuple.put(fieldId, DatumFactory.createNullDatum());
+ }
+ } else if ("space_name".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(tablespace.getSpaceName()));
+ } else if ("space_handler".equalsIgnoreCase(column.getSimpleName())) {
+ if (tablespace.hasHandler()) {
+ aTuple.put(fieldId, DatumFactory.createText(tablespace.getHandler()));
+ } else {
+ aTuple.put(fieldId, DatumFactory.createNullDatum());
+ }
+ } else if ("space_uri".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(tablespace.getUri()));
+ }
+ }
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
+
+ private List<Tuple> getDatabases(Schema outSchema) {
+ List<DatabaseProto> databases = masterContext.getCatalog().getAllDatabases();
+ List<Tuple> tuples = new ArrayList<Tuple>(databases.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+
+ for (DatabaseProto database: databases) {
+ aTuple = new VTuple(outSchema.size());
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column column = columns.get(fieldId);
+ if ("db_id".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(database.getId()));
+ } else if ("db_name".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(database.getName()));
+ } else if ("space_id".equalsIgnoreCase(column.getSimpleName())) {
+ if (database.hasSpaceId()) {
+ aTuple.put(fieldId, DatumFactory.createInt4(database.getSpaceId()));
+ } else {
+ aTuple.put(fieldId, DatumFactory.createNullDatum());
+ }
+ }
+ }
+
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
+
+ private List<Tuple> getTables(Schema outSchema) {
+ List<TableDescriptorProto> tables = masterContext.getCatalog().getAllTables();
+ List<Tuple> tuples = new ArrayList<Tuple>(tables.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+
+ for (TableDescriptorProto table: tables) {
+ aTuple = new VTuple(outSchema.size());
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column column = columns.get(fieldId);
+ if ("tid".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(table.getTid()));
+ } else if ("db_id".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(table.getDbId()));
+ } else if ("table_name".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(table.getName()));
+ } else if ("table_type".equalsIgnoreCase(column.getSimpleName())) {
+ if (table.hasTableType()) {
+ aTuple.put(fieldId, DatumFactory.createText(table.getTableType()));
+ } else {
+ aTuple.put(fieldId, DatumFactory.createNullDatum());
+ }
+ } else if ("path".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(table.getPath()));
+ } else if ("store_type".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(table.getStoreType()));
+ }
+ }
+
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
+
+ private List<Tuple> getColumns(Schema outSchema) {
+ List<ColumnProto> columnsList = masterContext.getCatalog().getAllColumns();
+ List<Tuple> tuples = new ArrayList<Tuple>(columnsList.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+ int columnId = 1, prevtid = -1, tid = 0;
+
+ for (ColumnProto column: columnsList) {
+ aTuple = new VTuple(outSchema.size());
+
+ tid = column.getTid();
+ if (prevtid != tid) {
+ columnId = 1;
+ prevtid = tid;
+ }
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column colObj = columns.get(fieldId);
+
+ if ("tid".equalsIgnoreCase(colObj.getSimpleName())) {
+ if (column.hasTid()) {
+ aTuple.put(fieldId, DatumFactory.createInt4(tid));
+ } else {
+ aTuple.put(fieldId, DatumFactory.createNullDatum());
+ }
+ } else if ("column_name".equalsIgnoreCase(colObj.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(column.getName()));
+ } else if ("ordinal_position".equalsIgnoreCase(colObj.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(columnId));
+ } else if ("data_type".equalsIgnoreCase(colObj.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(column.getDataType().getType().toString()));
+ } else if ("type_length".equalsIgnoreCase(colObj.getSimpleName())) {
+ DataType dataType = column.getDataType();
+ if (dataType.hasLength()) {
+ aTuple.put(fieldId, DatumFactory.createInt4(dataType.getLength()));
+ } else {
+ aTuple.put(fieldId, DatumFactory.createNullDatum());
+ }
+ }
+ }
+
+ columnId++;
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
-
++
+ private List<Tuple> getIndexes(Schema outSchema) {
- List<IndexProto> indexList = masterContext.getCatalog().getAllIndexes();
++ List<IndexDescProto> indexList = masterContext.getCatalog().getAllIndexes();
+ List<Tuple> tuples = new ArrayList<Tuple>(indexList.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
-
- for (IndexProto index: indexList) {
++
++ for (IndexDescProto index: indexList) {
+ aTuple = new VTuple(outSchema.size());
-
++
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column column = columns.get(fieldId);
-
++
+ if ("db_id".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt4(index.getDbId()));
++ aTuple.put(fieldId, DatumFactory.createInt4(index.getTableIdentifier().getDbId()));
+ } else if ("tid".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt4(index.getTId()));
++ aTuple.put(fieldId, DatumFactory.createInt4(index.getTableIdentifier().getTid()));
+ } else if ("index_name".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(index.getIndexName()));
- } else if ("column_name".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(index.getColumnName()));
- } else if ("data_type".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(index.getDataType()));
- } else if ("index_type".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(index.getIndexType()));
- } else if ("is_unique".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createBool(index.getIsUnique()));
- } else if ("is_clustered".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createBool(index.getIsClustered()));
- } else if ("is_ascending".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createBool(index.getIsAscending()));
++ } else if ("index_method".equalsIgnoreCase(column.getSimpleName())) {
++ aTuple.put(fieldId, DatumFactory.createText(index.getIndexMethod().name()));
++ } else if ("index_path".equalsIgnoreCase(column.getSimpleName())) {
++ aTuple.put(fieldId, DatumFactory.createText(index.getIndexPath()));
+ }
+ }
-
++
+ tuples.add(aTuple);
+ }
-
++
+ return tuples;
+ }
+
+ private List<Tuple> getAllTableOptions(Schema outSchema) {
+ List<TableOptionProto> optionList = masterContext.getCatalog().getAllTableOptions();
+ List<Tuple> tuples = new ArrayList<Tuple>(optionList.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+
+ for (TableOptionProto option: optionList) {
+ aTuple = new VTuple(outSchema.size());
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column column = columns.get(fieldId);
+
+ if ("tid".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(option.getTid()));
+ } else if ("key_".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(option.getKeyval().getKey()));
+ } else if ("value_".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(option.getKeyval().getValue()));
+ }
+ }
+
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
+
+ private List<Tuple> getAllTableStats(Schema outSchema) {
+ List<TableStatsProto> statList = masterContext.getCatalog().getAllTableStats();
+ List<Tuple> tuples = new ArrayList<Tuple>(statList.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+
+ for (TableStatsProto stat: statList) {
+ aTuple = new VTuple(outSchema.size());
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column column = columns.get(fieldId);
+
+ if ("tid".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(stat.getTid()));
+ } else if ("num_rows".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt8(stat.getNumRows()));
+ } else if ("num_bytes".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt8(stat.getNumBytes()));
+ }
+ }
+
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
+
+ private List<Tuple> getAllPartitions(Schema outSchema) {
+ List<TablePartitionProto> partitionList = masterContext.getCatalog().getAllPartitions();
+ List<Tuple> tuples = new ArrayList<Tuple>(partitionList.size());
+ List<Column> columns = outSchema.getColumns();
+ Tuple aTuple;
+
+ for (TablePartitionProto partition: partitionList) {
+ aTuple = new VTuple(outSchema.size());
+
+ for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
+ Column column = columns.get(fieldId);
+
+ if ("pid".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(partition.getPid()));
+ } else if ("tid".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(partition.getTid()));
+ } else if ("partition_name".equalsIgnoreCase(column.getSimpleName())) {
+ if (partition.hasPartitionName()) {
+ aTuple.put(fieldId, DatumFactory.createText(partition.getPartitionName()));
+ } else {
+ aTuple.put(fieldId, DatumFactory.createNullDatum());
+ }
+ } else if ("ordinal_position".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createInt4(partition.getOrdinalPosition()));
+ } else if ("path".equalsIgnoreCase(column.getSimpleName())) {
+ aTuple.put(fieldId, DatumFactory.createText(partition.getPath()));
+ }
+ }
+
+ tuples.add(aTuple);
+ }
+
+ return tuples;
+ }
+
+ private List<Tuple> fetchSystemTable(TableDesc tableDesc, Schema inSchema) {
+ List<Tuple> tuples = null;
+ String tableName = CatalogUtil.extractSimpleName(tableDesc.getName());
+
+ if ("tablespace".equalsIgnoreCase(tableName)) {
+ tuples = getTablespaces(inSchema);
+ } else if ("databases".equalsIgnoreCase(tableName)) {
+ tuples = getDatabases(inSchema);
+ } else if ("tables".equalsIgnoreCase(tableName)) {
+ tuples = getTables(inSchema);
+ } else if ("columns".equalsIgnoreCase(tableName)) {
+ tuples = getColumns(inSchema);
+ } else if ("indexes".equalsIgnoreCase(tableName)) {
+ tuples = getIndexes(inSchema);
+ } else if ("table_options".equalsIgnoreCase(tableName)) {
+ tuples = getAllTableOptions(inSchema);
+ } else if ("table_stats".equalsIgnoreCase(tableName)) {
+ tuples = getAllTableStats(inSchema);
+ } else if ("partitions".equalsIgnoreCase(tableName)) {
+ tuples = getAllPartitions(inSchema);
+ }
+
+ return tuples;
+ }
+
+ @Override
+ public List<ByteString> getNextRows(int fetchRowNum) throws IOException {
+ List<ByteString> rows = new ArrayList<ByteString>();
+ int startRow = currentRow;
+ int endRow = startRow + fetchRowNum;
+
+ if (physicalExec == null) {
+ return rows;
+ }
+
+ while (currentRow < endRow) {
+ Tuple currentTuple = physicalExec.next();
+
+ if (currentTuple == null) {
+ physicalExec.close();
+ physicalExec = null;
+ break;
+ }
+
+ currentRow++;
+ rows.add(ByteString.copyFrom(encoder.toBytes(currentTuple)));
+
+ if (currentRow >= maxRow) {
+ physicalExec.close();
+ physicalExec = null;
+ break;
+ }
+ }
+
+ return rows;
+ }
+
+ @Override
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ @Override
+ public String getSessionId() {
+ return sessionId;
+ }
+
+ @Override
+ public TableDesc getTableDesc() {
+ return tableDesc;
+ }
+
+ @Override
+ public Schema getLogicalSchema() {
+ return outSchema;
+ }
+
+ class SimplePhysicalPlannerImpl extends PhysicalPlannerImpl {
+
+ public SimplePhysicalPlannerImpl(TajoConf conf) {
+ super(conf);
+ }
+
+ @Override
+ public PhysicalExec createScanPlan(TaskAttemptContext ctx, ScanNode scanNode, Stack<LogicalNode> node)
+ throws IOException {
+ return new SystemPhysicalExec(ctx, scanNode);
+ }
+
+ @Override
+ public PhysicalExec createIndexScanExec(TaskAttemptContext ctx, IndexScanNode annotation) throws IOException {
+ return new SystemPhysicalExec(ctx, annotation);
+ }
+ }
+
+ class SystemPhysicalExec extends PhysicalExec {
+
+ private ScanNode scanNode;
+ private EvalNode qual;
+ private Projector projector;
+ private TableStats tableStats;
+ private final List<Tuple> cachedData;
+ private int currentRow;
+ private boolean isClosed;
+
+ public SystemPhysicalExec(TaskAttemptContext context, ScanNode scanNode) {
+ super(context, scanNode.getInSchema(), scanNode.getOutSchema());
+ this.scanNode = scanNode;
+ this.qual = this.scanNode.getQual();
+ cachedData = TUtil.newList();
+ currentRow = 0;
+ isClosed = false;
+
+ projector = new Projector(context, inSchema, outSchema, scanNode.getTargets());
+ }
+
+ @Override
+ public Tuple next() throws IOException {
+ Tuple aTuple = null;
+ Tuple outTuple = new VTuple(outColumnNum);
+
+ if (isClosed) {
+ return null;
+ }
+
+ if (cachedData.size() == 0) {
+ rescan();
+ }
+
+ if (!scanNode.hasQual()) {
+ if (currentRow < cachedData.size()) {
+ aTuple = cachedData.get(currentRow++);
+ projector.eval(aTuple, outTuple);
+ outTuple.setOffset(aTuple.getOffset());
+ return outTuple;
+ }
+ return null;
+ } else {
+ while (currentRow < cachedData.size()) {
+ aTuple = cachedData.get(currentRow++);
+ if (qual.eval(inSchema, aTuple).isTrue()) {
+ projector.eval(aTuple, outTuple);
+ return outTuple;
+ }
+ }
+ return null;
+ }
+ }
+
+ @Override
+ public void rescan() throws IOException {
+ cachedData.clear();
+ cachedData.addAll(fetchSystemTable(scanNode.getTableDesc(), inSchema));
+
+ tableStats = new TableStats();
+ tableStats.setNumRows(cachedData.size());
+ }
+
+ @Override
+ public void close() throws IOException {
+ scanNode = null;
+ qual = null;
+ projector = null;
+ cachedData.clear();
+ currentRow = -1;
+ isClosed = true;
+ }
+
+ @Override
+ public float getProgress() {
+ return 1.0f;
+ }
+
+ @Override
+ protected void compile() throws CompilationError {
+ if (scanNode.hasQual()) {
+ qual = context.getPrecompiledEval(inSchema, qual);
+ }
+ }
+
+ @Override
+ public TableStats getInputStats() {
+ return tableStats;
+ }
+
+ }
+
+ }
http://git-wip-us.apache.org/repos/asf/tajo/blob/e04c65fd/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
index 26476a3,2fbebc1..e7a3cf7
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java
@@@ -42,12 -40,8 +42,9 @@@ import org.apache.tajo.engine.planner.p
import org.apache.tajo.engine.planner.physical.StoreTableExec;
import org.apache.tajo.engine.query.QueryContext;
import org.apache.tajo.ipc.ClientProtos;
+import org.apache.tajo.ipc.ClientProtos.ResultCode;
import org.apache.tajo.ipc.ClientProtos.SubmitQueryResponse;
- import org.apache.tajo.master.NonForwardQueryResultFileScanner;
- import org.apache.tajo.master.NonForwardQueryResultScanner;
- import org.apache.tajo.master.NonForwardQueryResultSystemScanner;
- import org.apache.tajo.master.TajoMaster;
+ import org.apache.tajo.master.*;
import org.apache.tajo.master.exec.prehook.CreateTableHook;
import org.apache.tajo.master.exec.prehook.DistributedQueryHookManager;
import org.apache.tajo.master.exec.prehook.InsertIntoHook;
http://git-wip-us.apache.org/repos/asf/tajo/blob/e04c65fd/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
----------------------------------------------------------------------
diff --cc tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
index 0000000,2932694..4a8c188
mode 000000,100644..100644
--- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
@@@ -1,0 -1,738 +1,783 @@@
+ /**
+ * 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.tajo.querymaster;
+
+ import com.google.common.collect.Maps;
+ import org.apache.commons.lang.exception.ExceptionUtils;
+ import org.apache.commons.logging.Log;
+ import org.apache.commons.logging.LogFactory;
+ import org.apache.hadoop.fs.ContentSummary;
+ import org.apache.hadoop.fs.FileSystem;
+ import org.apache.hadoop.fs.Path;
+ import org.apache.hadoop.yarn.event.EventHandler;
+ import org.apache.hadoop.yarn.state.*;
+ import org.apache.hadoop.yarn.util.Clock;
+ import org.apache.tajo.ExecutionBlockId;
+ import org.apache.tajo.QueryId;
+ import org.apache.tajo.SessionVars;
+ import org.apache.tajo.TajoProtos.QueryState;
++import org.apache.tajo.catalog.*;
++import org.apache.tajo.catalog.exception.CatalogException;
+ import org.apache.tajo.catalog.proto.CatalogProtos.UpdateTableStatsProto;
+ import org.apache.tajo.catalog.CatalogService;
+ import org.apache.tajo.catalog.TableDesc;
+ import org.apache.tajo.catalog.TableMeta;
+ import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+ import org.apache.tajo.catalog.statistics.TableStats;
+ import org.apache.tajo.conf.TajoConf;
+ import org.apache.tajo.engine.planner.global.ExecutionBlock;
+ import org.apache.tajo.engine.planner.global.ExecutionBlockCursor;
+ import org.apache.tajo.engine.planner.global.MasterPlan;
+ import org.apache.tajo.plan.logical.*;
+ import org.apache.tajo.engine.query.QueryContext;
+ import org.apache.tajo.master.event.*;
+ import org.apache.tajo.plan.util.PlannerUtil;
+ import org.apache.tajo.storage.StorageManager;
+ import org.apache.tajo.storage.StorageConstants;
+ import org.apache.tajo.util.TUtil;
+ import org.apache.tajo.util.history.QueryHistory;
+ import org.apache.tajo.util.history.StageHistory;
+
+ import java.io.IOException;
+ import java.util.*;
+ import java.util.concurrent.locks.Lock;
+ import java.util.concurrent.locks.ReadWriteLock;
+ import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+ public class Query implements EventHandler<QueryEvent> {
+ private static final Log LOG = LogFactory.getLog(Query.class);
+
+ // Facilities for Query
+ private final TajoConf systemConf;
+ private final Clock clock;
+ private String queryStr;
+ private Map<ExecutionBlockId, Stage> stages;
+ private final EventHandler eventHandler;
+ private final MasterPlan plan;
+ QueryMasterTask.QueryMasterTaskContext context;
+ private ExecutionBlockCursor cursor;
+
+ // Query Status
+ private final QueryId id;
+ private long appSubmitTime;
+ private long startTime;
+ private long finishTime;
+ private TableDesc resultDesc;
+ private int completedStagesCount = 0;
+ private int successedStagesCount = 0;
+ private int killedStagesCount = 0;
+ private int failedStagesCount = 0;
+ private int erroredStagesCount = 0;
+ private final List<String> diagnostics = new ArrayList<String>();
+
+ // Internal Variables
+ private final Lock readLock;
+ private final Lock writeLock;
+ private int priority = 100;
+
+ // State Machine
+ private final StateMachine<QueryState, QueryEventType, QueryEvent> stateMachine;
+ private QueryState queryState;
+
+ // Transition Handler
+ private static final SingleArcTransition INTERNAL_ERROR_TRANSITION = new InternalErrorTransition();
+ private static final DiagnosticsUpdateTransition DIAGNOSTIC_UPDATE_TRANSITION = new DiagnosticsUpdateTransition();
+ private static final StageCompletedTransition STAGE_COMPLETED_TRANSITION = new StageCompletedTransition();
+ private static final QueryCompletedTransition QUERY_COMPLETED_TRANSITION = new QueryCompletedTransition();
+
+ protected static final StateMachineFactory
+ <Query,QueryState,QueryEventType,QueryEvent> stateMachineFactory =
+ new StateMachineFactory<Query, QueryState, QueryEventType, QueryEvent>
+ (QueryState.QUERY_NEW)
+
+ // Transitions from NEW state
+ .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_RUNNING,
+ QueryEventType.START,
+ new StartTransition())
+ .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_NEW,
+ QueryEventType.DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_KILLED,
+ QueryEventType.KILL,
+ new KillNewQueryTransition())
+ .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_ERROR,
+ QueryEventType.INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+
+ // Transitions from RUNNING state
+ .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_RUNNING,
+ QueryEventType.STAGE_COMPLETED,
+ STAGE_COMPLETED_TRANSITION)
+ .addTransition(QueryState.QUERY_RUNNING,
+ EnumSet.of(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_FAILED, QueryState.QUERY_KILLED,
+ QueryState.QUERY_ERROR),
+ QueryEventType.QUERY_COMPLETED,
+ QUERY_COMPLETED_TRANSITION)
+ .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_RUNNING,
+ QueryEventType.DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_KILL_WAIT,
+ QueryEventType.KILL,
+ new KillAllStagesTransition())
+ .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_ERROR,
+ QueryEventType.INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+
+ // Transitions from QUERY_SUCCEEDED state
+ .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_SUCCEEDED,
+ QueryEventType.DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ // ignore-able transitions
+ .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_SUCCEEDED,
+ QueryEventType.STAGE_COMPLETED,
+ STAGE_COMPLETED_TRANSITION)
+ .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_SUCCEEDED,
+ QueryEventType.KILL)
+ .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_ERROR,
+ QueryEventType.INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+
+ // Transitions from KILL_WAIT state
+ .addTransition(QueryState.QUERY_KILL_WAIT, QueryState.QUERY_KILL_WAIT,
+ QueryEventType.STAGE_COMPLETED,
+ STAGE_COMPLETED_TRANSITION)
+ .addTransition(QueryState.QUERY_KILL_WAIT,
+ EnumSet.of(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_FAILED, QueryState.QUERY_KILLED,
+ QueryState.QUERY_ERROR),
+ QueryEventType.QUERY_COMPLETED,
+ QUERY_COMPLETED_TRANSITION)
+ .addTransition(QueryState.QUERY_KILL_WAIT, QueryState.QUERY_KILL_WAIT,
+ QueryEventType.DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(QueryState.QUERY_KILL_WAIT, QueryState.QUERY_ERROR,
+ QueryEventType.INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+ // Ignore-able transitions
+ .addTransition(QueryState.QUERY_KILL_WAIT, EnumSet.of(QueryState.QUERY_KILLED),
+ QueryEventType.KILL,
+ QUERY_COMPLETED_TRANSITION)
+
+ // Transitions from FAILED state
+ .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_FAILED,
+ QueryEventType.DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_ERROR,
+ QueryEventType.INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+ // Ignore-able transitions
+ .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_FAILED,
+ QueryEventType.KILL)
+
+ // Transitions from ERROR state
+ .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
+ QueryEventType.DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
+ QueryEventType.INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+ // Ignore-able transitions
+ .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
+ EnumSet.of(QueryEventType.KILL, QueryEventType.STAGE_COMPLETED))
+
+ .installTopology();
+
+ public Query(final QueryMasterTask.QueryMasterTaskContext context, final QueryId id,
+ final long appSubmitTime,
+ final String queryStr,
+ final EventHandler eventHandler,
+ final MasterPlan plan) {
+ this.context = context;
+ this.systemConf = context.getConf();
+ this.id = id;
+ this.clock = context.getClock();
+ this.appSubmitTime = appSubmitTime;
+ this.queryStr = queryStr;
+ this.stages = Maps.newConcurrentMap();
+ this.eventHandler = eventHandler;
+ this.plan = plan;
+ this.cursor = new ExecutionBlockCursor(plan, true);
+
+ StringBuilder sb = new StringBuilder("\n=======================================================");
+ sb.append("\nThe order of execution: \n");
+ int order = 1;
+ while (cursor.hasNext()) {
+ ExecutionBlock currentEB = cursor.nextBlock();
+ sb.append("\n").append(order).append(": ").append(currentEB.getId());
+ order++;
+ }
+ sb.append("\n=======================================================");
+ LOG.info(sb);
+ cursor.reset();
+
+ ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+ this.readLock = readWriteLock.readLock();
+ this.writeLock = readWriteLock.writeLock();
+
+ stateMachine = stateMachineFactory.make(this);
+ queryState = stateMachine.getCurrentState();
+ }
+
+ public float getProgress() {
+ QueryState state = getState();
+ if (state == QueryState.QUERY_SUCCEEDED) {
+ return 1.0f;
+ } else {
+ int idx = 0;
+ List<Stage> tempStages = new ArrayList<Stage>();
+ synchronized(stages) {
+ tempStages.addAll(stages.values());
+ }
+
+ float [] subProgresses = new float[tempStages.size()];
+ for (Stage stage: tempStages) {
+ if (stage.getState() != StageState.NEW) {
+ subProgresses[idx] = stage.getProgress();
+ } else {
+ subProgresses[idx] = 0.0f;
+ }
+ idx++;
+ }
+
+ float totalProgress = 0.0f;
+ float proportion = 1.0f / (float)(getExecutionBlockCursor().size() - 1); // minus one is due to
+
+ for (int i = 0; i < subProgresses.length; i++) {
+ totalProgress += subProgresses[i] * proportion;
+ }
+
+ return totalProgress;
+ }
+ }
+
+ public long getAppSubmitTime() {
+ return this.appSubmitTime;
+ }
+
+ public long getStartTime() {
+ return startTime;
+ }
+
+ public void setStartTime() {
+ startTime = clock.getTime();
+ }
+
+ public long getFinishTime() {
+ return finishTime;
+ }
+
+ public void setFinishTime() {
+ finishTime = clock.getTime();
+ }
+
+ public QueryHistory getQueryHistory() {
+ QueryHistory queryHistory = makeQueryHistory();
+ queryHistory.setStageHistories(makeStageHistories());
+ return queryHistory;
+ }
+
+ private List<StageHistory> makeStageHistories() {
+ List<StageHistory> stageHistories = new ArrayList<StageHistory>();
+ for(Stage eachStage : getStages()) {
+ stageHistories.add(eachStage.getStageHistory());
+ }
+
+ return stageHistories;
+ }
+
+ private QueryHistory makeQueryHistory() {
+ QueryHistory queryHistory = new QueryHistory();
+
+ queryHistory.setQueryId(getId().toString());
+ queryHistory.setQueryMaster(context.getQueryMasterContext().getWorkerContext().getWorkerName());
+ queryHistory.setHttpPort(context.getQueryMasterContext().getWorkerContext().getConnectionInfo().getHttpInfoPort());
+ queryHistory.setLogicalPlan(plan.toString());
+ queryHistory.setLogicalPlan(plan.getLogicalPlan().toString());
+ queryHistory.setDistributedPlan(plan.toString());
+
+ List<String[]> sessionVariables = new ArrayList<String[]>();
+ for(Map.Entry<String,String> entry: plan.getContext().getAllKeyValus().entrySet()) {
+ if (SessionVars.exists(entry.getKey()) && SessionVars.isPublic(SessionVars.get(entry.getKey()))) {
+ sessionVariables.add(new String[]{entry.getKey(), entry.getValue()});
+ }
+ }
+ queryHistory.setSessionVariables(sessionVariables);
+
+ return queryHistory;
+ }
+
+ public List<String> getDiagnostics() {
+ readLock.lock();
+ try {
+ return diagnostics;
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ protected void addDiagnostic(String diag) {
+ diagnostics.add(diag);
+ }
+
+ public TableDesc getResultDesc() {
+ return resultDesc;
+ }
+
+ public void setResultDesc(TableDesc desc) {
+ resultDesc = desc;
+ }
+
+ public MasterPlan getPlan() {
+ return plan;
+ }
+
+ public StateMachine<QueryState, QueryEventType, QueryEvent> getStateMachine() {
+ return stateMachine;
+ }
+
+ public void addStage(Stage stage) {
+ stages.put(stage.getId(), stage);
+ }
+
+ public QueryId getId() {
+ return this.id;
+ }
+
+ public Stage getStage(ExecutionBlockId id) {
+ return this.stages.get(id);
+ }
+
+ public Collection<Stage> getStages() {
+ return this.stages.values();
+ }
+
+ public QueryState getSynchronizedState() {
+ readLock.lock();
+ try {
+ return stateMachine.getCurrentState();
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ /* non-blocking call for client API */
+ public QueryState getState() {
+ return queryState;
+ }
+
+ public ExecutionBlockCursor getExecutionBlockCursor() {
+ return cursor;
+ }
+
+ public static class StartTransition
+ implements SingleArcTransition<Query, QueryEvent> {
+
+ @Override
+ public void transition(Query query, QueryEvent queryEvent) {
+
+ query.setStartTime();
+ Stage stage = new Stage(query.context, query.getPlan(),
+ query.getExecutionBlockCursor().nextBlock());
+ stage.setPriority(query.priority--);
+ query.addStage(stage);
+
+ stage.handle(new StageEvent(stage.getId(), StageEventType.SQ_INIT));
+ LOG.debug("Schedule unit plan: \n" + stage.getBlock().getPlan());
+ }
+ }
+
+ public static class QueryCompletedTransition implements MultipleArcTransition<Query, QueryEvent, QueryState> {
+
+ @Override
+ public QueryState transition(Query query, QueryEvent queryEvent) {
+ QueryCompletedEvent stageEvent = (QueryCompletedEvent) queryEvent;
+ QueryState finalState;
+
+ if (stageEvent.getState() == StageState.SUCCEEDED) {
+ finalState = finalizeQuery(query, stageEvent);
+ } else if (stageEvent.getState() == StageState.FAILED) {
+ finalState = QueryState.QUERY_FAILED;
+ } else if (stageEvent.getState() == StageState.KILLED) {
+ finalState = QueryState.QUERY_KILLED;
+ } else {
+ finalState = QueryState.QUERY_ERROR;
+ }
+ if (finalState != QueryState.QUERY_SUCCEEDED) {
+ Stage lastStage = query.getStage(stageEvent.getExecutionBlockId());
+ if (lastStage != null && lastStage.getTableMeta() != null) {
+ StoreType storeType = lastStage.getTableMeta().getStoreType();
+ if (storeType != null) {
+ LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
+ try {
+ StorageManager.getStorageManager(query.systemConf, storeType).rollbackOutputCommit(rootNode.getChild());
+ } catch (IOException e) {
+ LOG.warn(query.getId() + ", failed processing cleanup storage when query failed:" + e.getMessage(), e);
+ }
+ }
+ }
+ }
+ query.eventHandler.handle(new QueryMasterQueryCompletedEvent(query.getId()));
+ query.setFinishTime();
+
+ return finalState;
+ }
+
+ private QueryState finalizeQuery(Query query, QueryCompletedEvent event) {
+ Stage lastStage = query.getStage(event.getExecutionBlockId());
+ StoreType storeType = lastStage.getTableMeta().getStoreType();
+ try {
+ LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
+ CatalogService catalog = lastStage.getContext().getQueryMasterContext().getWorkerContext().getCatalog();
+ TableDesc tableDesc = PlannerUtil.getTableDesc(catalog, rootNode.getChild());
+
+ Path finalOutputDir = StorageManager.getStorageManager(query.systemConf, storeType)
+ .commitOutputData(query.context.getQueryContext(),
+ lastStage.getId(), lastStage.getMasterPlan().getLogicalPlan(), lastStage.getSchema(), tableDesc);
+
+ QueryHookExecutor hookExecutor = new QueryHookExecutor(query.context.getQueryMasterContext());
+ hookExecutor.execute(query.context.getQueryContext(), query, event.getExecutionBlockId(), finalOutputDir);
+ } catch (Exception e) {
+ query.eventHandler.handle(new QueryDiagnosticsUpdateEvent(query.id, ExceptionUtils.getStackTrace(e)));
+ return QueryState.QUERY_ERROR;
+ }
+
+ return QueryState.QUERY_SUCCEEDED;
+ }
+
+ private static interface QueryHook {
+ boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir);
+ void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext, Query query,
+ ExecutionBlockId finalExecBlockId, Path finalOutputDir) throws Exception;
+ }
+
+ private class QueryHookExecutor {
+ private List<QueryHook> hookList = TUtil.newList();
+ private QueryMaster.QueryMasterContext context;
+
+ public QueryHookExecutor(QueryMaster.QueryMasterContext context) {
+ this.context = context;
+ hookList.add(new MaterializedResultHook());
+ hookList.add(new CreateTableHook());
+ hookList.add(new InsertTableHook());
++ hookList.add(new CreateIndexHook());
+ }
+
+ public void execute(QueryContext queryContext, Query query,
+ ExecutionBlockId finalExecBlockId,
+ Path finalOutputDir) throws Exception {
+ for (QueryHook hook : hookList) {
+ if (hook.isEligible(queryContext, query, finalExecBlockId, finalOutputDir)) {
+ hook.execute(context, queryContext, query, finalExecBlockId, finalOutputDir);
+ }
+ }
+ }
++ }
++
++ private class CreateIndexHook implements QueryHook {
++
++ @Override
++ public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir) {
++ Stage lastStage = query.getStage(finalExecBlockId);
++ return lastStage.getBlock().getPlan().getType() == NodeType.CREATE_INDEX;
++ }
++
++ @Override
++ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir) throws Exception {
++ CatalogService catalog = context.getWorkerContext().getCatalog();
++ Stage lastStage = query.getStage(finalExecBlockId);
++
++ CreateIndexNode createIndexNode = (CreateIndexNode) lastStage.getBlock().getPlan();
++ String databaseName, simpleIndexName, qualifiedIndexName;
++ if (CatalogUtil.isFQTableName(createIndexNode.getIndexName())) {
++ String [] splits = CatalogUtil.splitFQTableName(createIndexNode.getIndexName());
++ databaseName = splits[0];
++ simpleIndexName = splits[1];
++ qualifiedIndexName = createIndexNode.getIndexName();
++ } else {
++ databaseName = queryContext.getCurrentDatabase();
++ simpleIndexName = createIndexNode.getIndexName();
++ qualifiedIndexName = CatalogUtil.buildFQName(databaseName, simpleIndexName);
++ }
++ ScanNode scanNode = PlannerUtil.findTopNode(createIndexNode, NodeType.SCAN);
++ if (scanNode == null) {
++ throw new IOException("Cannot find the table of the relation");
++ }
++ IndexDesc indexDesc = new IndexDesc(databaseName, scanNode.getTableName(),
++ simpleIndexName, createIndexNode.getIndexPath(),
++ createIndexNode.getKeySortSpecs(), createIndexNode.getIndexMethod(),
++ createIndexNode.isUnique(), false, scanNode.getLogicalSchema());
++ if (catalog.createIndex(indexDesc)) {
++ LOG.info("Index " + qualifiedIndexName + " is created for the table " + scanNode.getTableName() + ".");
++ } else {
++ LOG.info("Index creation " + qualifiedIndexName + " is failed.");
++ throw new CatalogException("Cannot create index \"" + qualifiedIndexName + "\".");
++ }
++ }
+ }
+
+ private class MaterializedResultHook implements QueryHook {
+
+ @Override
+ public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
+ Path finalOutputDir) {
+ Stage lastStage = query.getStage(finalExecBlockId);
+ NodeType type = lastStage.getBlock().getPlan().getType();
+ return type != NodeType.CREATE_TABLE && type != NodeType.INSERT;
+ }
+
+ @Override
+ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext,
+ Query query, ExecutionBlockId finalExecBlockId,
+ Path finalOutputDir) throws Exception {
+ Stage lastStage = query.getStage(finalExecBlockId);
+ TableMeta meta = lastStage.getTableMeta();
+
+ String nullChar = queryContext.get(SessionVars.NULL_CHAR);
+ meta.putOption(StorageConstants.TEXT_NULL, nullChar);
+
+ TableStats stats = lastStage.getResultStats();
+
+ TableDesc resultTableDesc =
+ new TableDesc(
+ query.getId().toString(),
+ lastStage.getSchema(),
+ meta,
+ finalOutputDir.toUri());
+ resultTableDesc.setExternal(true);
+
+ stats.setNumBytes(getTableVolume(query.systemConf, finalOutputDir));
+ resultTableDesc.setStats(stats);
+ query.setResultDesc(resultTableDesc);
+ }
+ }
+
+ private class CreateTableHook implements QueryHook {
+
+ @Override
+ public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
+ Path finalOutputDir) {
+ Stage lastStage = query.getStage(finalExecBlockId);
+ return lastStage.getBlock().getPlan().getType() == NodeType.CREATE_TABLE;
+ }
+
+ @Override
+ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext,
+ Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir) throws Exception {
+ CatalogService catalog = context.getWorkerContext().getCatalog();
+ Stage lastStage = query.getStage(finalExecBlockId);
+ TableStats stats = lastStage.getResultStats();
+
+ CreateTableNode createTableNode = (CreateTableNode) lastStage.getBlock().getPlan();
+ TableMeta meta = new TableMeta(createTableNode.getStorageType(), createTableNode.getOptions());
+
+ TableDesc tableDescTobeCreated =
+ new TableDesc(
+ createTableNode.getTableName(),
+ createTableNode.getTableSchema(),
+ meta,
+ finalOutputDir.toUri());
+ tableDescTobeCreated.setExternal(createTableNode.isExternal());
+
+ if (createTableNode.hasPartition()) {
+ tableDescTobeCreated.setPartitionMethod(createTableNode.getPartitionMethod());
+ }
+
+ stats.setNumBytes(getTableVolume(query.systemConf, finalOutputDir));
+ tableDescTobeCreated.setStats(stats);
+ query.setResultDesc(tableDescTobeCreated);
+
+ catalog.createTable(tableDescTobeCreated);
+ }
+ }
+
+ private class InsertTableHook implements QueryHook {
+
+ @Override
+ public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
+ Path finalOutputDir) {
+ Stage lastStage = query.getStage(finalExecBlockId);
+ return lastStage.getBlock().getPlan().getType() == NodeType.INSERT;
+ }
+
+ @Override
+ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext,
+ Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir)
+ throws Exception {
+
+ CatalogService catalog = context.getWorkerContext().getCatalog();
+ Stage lastStage = query.getStage(finalExecBlockId);
+ TableMeta meta = lastStage.getTableMeta();
+ TableStats stats = lastStage.getResultStats();
+
+ InsertNode insertNode = (InsertNode) lastStage.getBlock().getPlan();
+
+ TableDesc finalTable;
+ if (insertNode.hasTargetTable()) {
+ String tableName = insertNode.getTableName();
+ finalTable = catalog.getTableDesc(tableName);
+ } else {
+ String tableName = query.getId().toString();
+ finalTable = new TableDesc(tableName, lastStage.getSchema(), meta, finalOutputDir.toUri());
+ }
+
+ long volume = getTableVolume(query.systemConf, finalOutputDir);
+ stats.setNumBytes(volume);
+ finalTable.setStats(stats);
+
+ if (insertNode.hasTargetTable()) {
+ UpdateTableStatsProto.Builder builder = UpdateTableStatsProto.newBuilder();
+ builder.setTableName(finalTable.getName());
+ builder.setStats(stats.getProto());
+
+ catalog.updateTableStats(builder.build());
+ }
+
+ query.setResultDesc(finalTable);
+ }
+ }
+ }
+
+ public static long getTableVolume(TajoConf systemConf, Path tablePath) throws IOException {
+ FileSystem fs = tablePath.getFileSystem(systemConf);
+ ContentSummary directorySummary = fs.getContentSummary(tablePath);
+ return directorySummary.getLength();
+ }
+
+ public static class StageCompletedTransition implements SingleArcTransition<Query, QueryEvent> {
+
+ private boolean hasNext(Query query) {
+ ExecutionBlockCursor cursor = query.getExecutionBlockCursor();
+ ExecutionBlock nextBlock = cursor.peek();
+ return !query.getPlan().isTerminal(nextBlock);
+ }
+
+ private void executeNextBlock(Query query) {
+ ExecutionBlockCursor cursor = query.getExecutionBlockCursor();
+ ExecutionBlock nextBlock = cursor.nextBlock();
+ Stage nextStage = new Stage(query.context, query.getPlan(), nextBlock);
+ nextStage.setPriority(query.priority--);
+ query.addStage(nextStage);
+ nextStage.handle(new StageEvent(nextStage.getId(), StageEventType.SQ_INIT));
+
+ LOG.info("Scheduling Stage:" + nextStage.getId());
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("Scheduling Stage's Priority: " + nextStage.getPriority());
+ LOG.debug("Scheduling Stage's Plan: \n" + nextStage.getBlock().getPlan());
+ }
+ }
+
+ @Override
+ public void transition(Query query, QueryEvent event) {
+ try {
+ query.completedStagesCount++;
+ StageCompletedEvent castEvent = (StageCompletedEvent) event;
+
+ if (castEvent.getState() == StageState.SUCCEEDED) {
+ query.successedStagesCount++;
+ } else if (castEvent.getState() == StageState.KILLED) {
+ query.killedStagesCount++;
+ } else if (castEvent.getState() == StageState.FAILED) {
+ query.failedStagesCount++;
+ } else if (castEvent.getState() == StageState.ERROR) {
+ query.erroredStagesCount++;
+ } else {
+ LOG.error(String.format("Invalid Stage (%s) State %s at %s",
+ castEvent.getExecutionBlockId().toString(), castEvent.getState().name(), query.getSynchronizedState().name()));
+ query.eventHandler.handle(new QueryEvent(event.getQueryId(), QueryEventType.INTERNAL_ERROR));
+ }
+
+ // if a stage is succeeded and a query is running
+ if (castEvent.getState() == StageState.SUCCEEDED && // latest stage succeeded
+ query.getSynchronizedState() == QueryState.QUERY_RUNNING && // current state is not in KILL_WAIT, FAILED, or ERROR.
+ hasNext(query)) { // there remains at least one stage.
+ query.getStage(castEvent.getExecutionBlockId()).waitingIntermediateReport();
+ executeNextBlock(query);
+ } else { // if a query is completed due to finished, kill, failure, or error
+ query.eventHandler.handle(new QueryCompletedEvent(castEvent.getExecutionBlockId(), castEvent.getState()));
+ }
+ } catch (Throwable t) {
+ LOG.error(t.getMessage(), t);
+ query.eventHandler.handle(new QueryEvent(event.getQueryId(), QueryEventType.INTERNAL_ERROR));
+ }
+ }
+ }
+
+ private static class DiagnosticsUpdateTransition implements SingleArcTransition<Query, QueryEvent> {
+ @Override
+ public void transition(Query query, QueryEvent event) {
+ query.addDiagnostic(((QueryDiagnosticsUpdateEvent) event).getDiagnosticUpdate());
+ }
+ }
+
+ private static class KillNewQueryTransition implements SingleArcTransition<Query, QueryEvent> {
+ @Override
+ public void transition(Query query, QueryEvent event) {
+ query.setFinishTime();
+ query.eventHandler.handle(new QueryMasterQueryCompletedEvent(query.getId()));
+ }
+ }
+
+ private static class KillAllStagesTransition implements SingleArcTransition<Query, QueryEvent> {
+ @Override
+ public void transition(Query query, QueryEvent event) {
+ synchronized (query.stages) {
+ for (Stage stage : query.stages.values()) {
+ query.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_KILL));
+ }
+ }
+ }
+ }
+
+ private static class InternalErrorTransition implements SingleArcTransition<Query, QueryEvent> {
+
+ @Override
+ public void transition(Query query, QueryEvent event) {
+ query.setFinishTime();
+ query.eventHandler.handle(new QueryMasterQueryCompletedEvent(query.getId()));
+ }
+ }
+
+ @Override
+ public void handle(QueryEvent event) {
+ LOG.info("Processing " + event.getQueryId() + " of type " + event.getType());
+ try {
+ writeLock.lock();
+ QueryState oldState = getSynchronizedState();
+ try {
+ getStateMachine().doTransition(event.getType(), event);
+ queryState = getSynchronizedState();
+ } catch (InvalidStateTransitonException e) {
+ LOG.error("Can't handle this event at current state"
+ + ", type:" + event
+ + ", oldState:" + oldState.name()
+ + ", nextState:" + getSynchronizedState().name()
+ , e);
+ eventHandler.handle(new QueryEvent(this.id, QueryEventType.INTERNAL_ERROR));
+ }
+
+ //notify the eventhandler of state change
+ if (oldState != getSynchronizedState()) {
+ LOG.info(id + " Query Transitioned from " + oldState + " to " + getSynchronizedState());
+ }
+ }
+
+ finally {
+ writeLock.unlock();
+ }
+ }
+ }
[11/16] tajo git commit: TAJO-1288: Refactoring
org.apache.tajo.master package.
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
deleted file mode 100644
index 559fc14..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryInfo.java
+++ /dev/null
@@ -1,235 +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.tajo.master.querymaster;
-
-
-import com.google.gson.annotations.Expose;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.TajoProtos;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.engine.json.CoreGsonHelper;
-import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.ipc.ClientProtos.QueryInfoProto;
-import org.apache.tajo.json.GsonObject;
-import org.apache.tajo.util.TajoIdUtils;
-import org.apache.tajo.util.history.History;
-
-public class QueryInfo implements GsonObject, History {
- private QueryId queryId;
- @Expose
- private QueryContext context;
- @Expose
- private String sql;
- @Expose
- private volatile TajoProtos.QueryState queryState;
- @Expose
- private volatile float progress;
- @Expose
- private volatile long startTime;
- @Expose
- private volatile long finishTime;
- @Expose
- private String lastMessage;
- @Expose
- private String hostNameOfQM;
- @Expose
- private int queryMasterPort;
- @Expose
- private int queryMasterClientPort;
- @Expose
- private int queryMasterInfoPort;
- @Expose
- private String queryIdStr;
- @Expose
- private volatile TableDesc resultDesc;
-
- private String jsonExpr;
-
- public QueryInfo(QueryId queryId) {
- this(queryId, null, null, null);
- }
-
- public QueryInfo(QueryId queryId, QueryContext queryContext, String sql, String jsonExpr) {
- this.queryId = queryId;
- this.queryIdStr = queryId.toString();
- this.context = queryContext;
- this.sql = sql;
- this.jsonExpr = jsonExpr;
-
- this.queryState = TajoProtos.QueryState.QUERY_MASTER_INIT;
- }
-
- public QueryId getQueryId() {
- return queryId;
- }
-
- public QueryContext getQueryContext() {
- return context;
- }
-
- public String getSql() {
- return sql;
- }
-
- public String getQueryMasterHost() {
- return hostNameOfQM;
- }
-
- public void setQueryMaster(String hostName) {
- this.hostNameOfQM = hostName;
- }
-
- public int getQueryMasterInfoPort() {
- return queryMasterInfoPort;
- }
-
- public void setQueryMasterInfoPort(int queryMasterInfoPort) {
- this.queryMasterInfoPort = queryMasterInfoPort;
- }
-
- public void setQueryMasterPort(int port) {
- this.queryMasterPort = port;
- }
-
- public int getQueryMasterPort() {
- return queryMasterPort;
- }
-
- public void setQueryMasterclientPort(int port) {
- queryMasterClientPort = port;
- }
-
- public int getQueryMasterClientPort() {
- return queryMasterClientPort;
- }
-
- public TajoProtos.QueryState getQueryState() {
- return queryState;
- }
-
- public void setQueryState(TajoProtos.QueryState queryState) {
- this.queryState = queryState;
- }
-
- public long getStartTime() {
- return startTime;
- }
-
- public void setStartTime(long startTime) {
- this.startTime = startTime;
- }
-
- public long getFinishTime() {
- return finishTime;
- }
-
- public void setFinishTime(long finishTime) {
- this.finishTime = finishTime;
- }
-
- public String getLastMessage() {
- return lastMessage;
- }
-
- public void setLastMessage(String lastMessage) {
- this.lastMessage = lastMessage;
- }
-
- public float getProgress() {
- return progress;
- }
-
- public void setProgress(float progress) {
- this.progress = progress;
- }
-
- public void setResultDesc(TableDesc result) {
- this.resultDesc = result;
- }
-
- public boolean hasResultdesc() {
- return resultDesc != null;
- }
-
- public TableDesc getResultDesc() {
- return resultDesc;
- }
-
- @Override
- public String toString() {
- return queryId.toString() + ",state=" + queryState +",progress=" + progress + ", queryMaster="
- + getQueryMasterHost();
- }
-
- public String getJsonExpr() {
- return jsonExpr;
- }
-
- @Override
- public String toJson() {
- return CoreGsonHelper.toJson(this, QueryInfo.class);
- }
-
- @Override
- public HistoryType getHistoryType() {
- return HistoryType.QUERY_SUMMARY;
- }
-
- public static QueryInfo fromJson(String json) {
- QueryInfo queryInfo = CoreGsonHelper.fromJson(json, QueryInfo.class);
- queryInfo.queryId = TajoIdUtils.parseQueryId(queryInfo.queryIdStr);
- return queryInfo;
- }
-
- public String getQueryIdStr() {
- return queryIdStr;
- }
-
- public QueryInfoProto getProto() {
- QueryInfoProto.Builder builder = QueryInfoProto.newBuilder();
-
- builder.setQueryId(queryId.toString())
- .setQueryState(queryState)
- .setContextVars(context.getProto())
- .setProgress(progress)
- .setStartTime(startTime)
- .setFinishTime(finishTime)
- .setQueryMasterPort(queryMasterPort)
- .setQueryMasterClientPort(queryMasterClientPort)
- .setQueryMasterInfoPort(queryMasterInfoPort);
-
- if (resultDesc != null) {
- builder.setResultDesc(resultDesc.getProto());
- }
-
- if (sql != null) {
- builder.setSql(sql);
- }
-
- if (lastMessage != null) {
- builder.setLastMessage(lastMessage);
- }
-
- if (hostNameOfQM != null) {
- builder.setHostNameOfQM(hostNameOfQM);
- }
-
- return builder.build();
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java
deleted file mode 100644
index ce30ec7..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobEvent.java
+++ /dev/null
@@ -1,45 +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.tajo.master.querymaster;
-
-import org.apache.hadoop.yarn.event.AbstractEvent;
-
-public class QueryJobEvent extends AbstractEvent<QueryJobEvent.Type> {
- private QueryInfo queryInfo;
-
- public QueryJobEvent(Type type, QueryInfo queryInfo) {
- super(type);
-
- this.queryInfo = queryInfo;
- }
-
- public QueryInfo getQueryInfo() {
- return this.queryInfo;
- }
-
- public enum Type {
- QUERY_JOB_START,
- QUERY_JOB_HEARTBEAT,
- QUERY_JOB_FINISH,
- QUERY_JOB_STOP,
- QUERY_MASTER_START,
- QUERY_MASTER_STOP,
- QUERY_JOB_KILL
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java
deleted file mode 100644
index 13f6456..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryJobManager.java
+++ /dev/null
@@ -1,310 +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.tajo.master.querymaster;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.CompositeService;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.QueryIdFactory;
-import org.apache.tajo.TajoProtos;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.ipc.TajoMasterProtocol;
-import org.apache.tajo.master.TajoMaster;
-import org.apache.tajo.master.cluster.WorkerConnectionInfo;
-import org.apache.tajo.master.session.Session;
-import org.apache.tajo.plan.logical.LogicalRootNode;
-import org.apache.tajo.scheduler.SimpleFifoScheduler;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicLong;
-
-public class QueryJobManager extends CompositeService {
- private static final Log LOG = LogFactory.getLog(QueryJobManager.class.getName());
-
- // TajoMaster Context
- private final TajoMaster.MasterContext masterContext;
-
- private AsyncDispatcher dispatcher;
-
- private SimpleFifoScheduler scheduler;
-
- private final Map<QueryId, QueryInProgress> submittedQueries = Maps.newConcurrentMap();
-
- private final Map<QueryId, QueryInProgress> runningQueries = Maps.newConcurrentMap();
-
- private AtomicLong minExecutionTime = new AtomicLong(Long.MAX_VALUE);
- private AtomicLong maxExecutionTime = new AtomicLong();
- private AtomicLong avgExecutionTime = new AtomicLong();
- private AtomicLong executedQuerySize = new AtomicLong();
-
- public QueryJobManager(final TajoMaster.MasterContext masterContext) {
- super(QueryJobManager.class.getName());
- this.masterContext = masterContext;
- }
-
- @Override
- public void init(Configuration conf) {
- try {
- this.dispatcher = new AsyncDispatcher();
- addService(this.dispatcher);
-
- this.dispatcher.register(QueryJobEvent.Type.class, new QueryJobManagerEventHandler());
-
- this.scheduler = new SimpleFifoScheduler(this);
- } catch (Exception e) {
- catchException(null, e);
- }
-
- super.init(conf);
- }
-
- @Override
- public void stop() {
- synchronized(runningQueries) {
- for(QueryInProgress eachQueryInProgress: runningQueries.values()) {
- eachQueryInProgress.stop();
- }
- }
- this.scheduler.stop();
- super.stop();
- }
-
- @Override
- public void start() {
- this.scheduler.start();
- super.start();
- }
-
- public EventHandler getEventHandler() {
- return dispatcher.getEventHandler();
- }
-
- public Collection<QueryInProgress> getSubmittedQueries() {
- synchronized (submittedQueries){
- return Collections.unmodifiableCollection(submittedQueries.values());
- }
- }
-
- public Collection<QueryInProgress> getRunningQueries() {
- synchronized (runningQueries){
- return Collections.unmodifiableCollection(runningQueries.values());
- }
- }
-
- public synchronized Collection<QueryInfo> getFinishedQueries() {
- try {
- return this.masterContext.getHistoryReader().getQueries(null);
- } catch (Throwable e) {
- LOG.error(e);
- return Lists.newArrayList();
- }
- }
-
-
- public synchronized QueryInfo getFinishedQuery(QueryId queryId) {
- try {
- return this.masterContext.getHistoryReader().getQueryInfo(queryId.toString());
- } catch (Throwable e) {
- LOG.error(e);
- return null;
- }
- }
-
- public QueryInfo createNewQueryJob(Session session, QueryContext queryContext, String sql,
- String jsonExpr, LogicalRootNode plan)
- throws Exception {
- QueryId queryId = QueryIdFactory.newQueryId(masterContext.getResourceManager().getSeedQueryId());
- QueryInProgress queryInProgress = new QueryInProgress(masterContext, session, queryContext, queryId, sql,
- jsonExpr, plan);
-
- synchronized (submittedQueries) {
- queryInProgress.getQueryInfo().setQueryMaster("");
- submittedQueries.put(queryInProgress.getQueryId(), queryInProgress);
- }
-
- scheduler.addQuery(queryInProgress);
- return queryInProgress.getQueryInfo();
- }
-
- public QueryInfo startQueryJob(QueryId queryId) throws Exception {
-
- QueryInProgress queryInProgress;
-
- synchronized (submittedQueries) {
- queryInProgress = submittedQueries.remove(queryId);
- }
-
- synchronized (runningQueries) {
- runningQueries.put(queryInProgress.getQueryId(), queryInProgress);
- }
-
- addService(queryInProgress);
- queryInProgress.init(getConfig());
- queryInProgress.start();
-
- if (!queryInProgress.startQueryMaster()) {
- stopQuery(queryId);
- }
-
- return queryInProgress.getQueryInfo();
- }
-
- public TajoMaster.MasterContext getMasterContext() {
- return masterContext;
- }
-
- class QueryJobManagerEventHandler implements EventHandler<QueryJobEvent> {
- @Override
- public void handle(QueryJobEvent event) {
- QueryInProgress queryInProgress = getQueryInProgress(event.getQueryInfo().getQueryId());
- if(queryInProgress == null) {
- LOG.warn("No query info in running queries.[" + event.getQueryInfo().getQueryId() + "]");
- return;
- }
-
- if (event.getType() == QueryJobEvent.Type.QUERY_JOB_STOP) {
- stopQuery(event.getQueryInfo().getQueryId());
- } else if (queryInProgress.isStarted()) {
- queryInProgress.getEventHandler().handle(event);
- } else if (event.getType() == QueryJobEvent.Type.QUERY_JOB_KILL) {
- scheduler.removeQuery(queryInProgress.getQueryId());
- queryInProgress.getQueryInfo().setQueryState(TajoProtos.QueryState.QUERY_KILLED);
-
- stopQuery(queryInProgress.getQueryId());
- }
- }
- }
-
- public QueryInProgress getQueryInProgress(QueryId queryId) {
- QueryInProgress queryInProgress;
- synchronized (submittedQueries) {
- queryInProgress = submittedQueries.get(queryId);
- }
-
- if (queryInProgress == null) {
- synchronized (runningQueries) {
- queryInProgress = runningQueries.get(queryId);
- }
- }
- return queryInProgress;
- }
-
- public void stopQuery(QueryId queryId) {
- LOG.info("Stop QueryInProgress:" + queryId);
- QueryInProgress queryInProgress = getQueryInProgress(queryId);
- if(queryInProgress != null) {
- queryInProgress.stop();
- synchronized(submittedQueries) {
- submittedQueries.remove(queryId);
- }
-
- synchronized(runningQueries) {
- runningQueries.remove(queryId);
- }
-
- QueryInfo queryInfo = queryInProgress.getQueryInfo();
- long executionTime = queryInfo.getFinishTime() - queryInfo.getStartTime();
- if (executionTime < minExecutionTime.get()) {
- minExecutionTime.set(executionTime);
- }
-
- if (executionTime > maxExecutionTime.get()) {
- maxExecutionTime.set(executionTime);
- }
-
- long totalExecutionTime = executedQuerySize.get() * avgExecutionTime.get();
- if (totalExecutionTime > 0) {
- avgExecutionTime.set((totalExecutionTime + executionTime) / (executedQuerySize.get() + 1));
- } else {
- avgExecutionTime.set(executionTime);
- }
- executedQuerySize.incrementAndGet();
- removeService(queryInProgress);
- } else {
- LOG.warn("No QueryInProgress while query stopping: " + queryId);
- }
- }
-
- public long getMinExecutionTime() {
- if (getExecutedQuerySize() == 0) return 0;
- return minExecutionTime.get();
- }
-
- public long getMaxExecutionTime() {
- return maxExecutionTime.get();
- }
-
- public long getAvgExecutionTime() {
- return avgExecutionTime.get();
- }
-
- public long getExecutedQuerySize() {
- return executedQuerySize.get();
- }
-
- private void catchException(QueryId queryId, Exception e) {
- LOG.error(e.getMessage(), e);
- QueryInProgress queryInProgress = runningQueries.get(queryId);
- queryInProgress.catchException(e);
- }
-
- public synchronized TajoMasterProtocol.TajoHeartbeatResponse.ResponseCommand queryHeartbeat(
- TajoMasterProtocol.TajoHeartbeat queryHeartbeat) {
- QueryInProgress queryInProgress = getQueryInProgress(new QueryId(queryHeartbeat.getQueryId()));
- if(queryInProgress == null) {
- return null;
- }
-
- QueryInfo queryInfo = makeQueryInfoFromHeartbeat(queryHeartbeat);
- getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_HEARTBEAT, queryInfo));
-
- return null;
- }
-
- private QueryInfo makeQueryInfoFromHeartbeat(TajoMasterProtocol.TajoHeartbeat queryHeartbeat) {
- QueryInfo queryInfo = new QueryInfo(new QueryId(queryHeartbeat.getQueryId()));
- WorkerConnectionInfo connectionInfo = new WorkerConnectionInfo(queryHeartbeat.getConnectionInfo());
-
- queryInfo.setQueryMaster(connectionInfo.getHost());
- queryInfo.setQueryMasterPort(connectionInfo.getQueryMasterPort());
- queryInfo.setQueryMasterclientPort(connectionInfo.getClientPort());
- queryInfo.setLastMessage(queryHeartbeat.getStatusMessage());
- queryInfo.setQueryState(queryHeartbeat.getState());
- queryInfo.setProgress(queryHeartbeat.getQueryProgress());
-
- if (queryHeartbeat.hasQueryFinishTime()) {
- queryInfo.setFinishTime(queryHeartbeat.getQueryFinishTime());
- }
-
- if (queryHeartbeat.hasResultDesc()) {
- queryInfo.setResultDesc(new TableDesc(queryHeartbeat.getResultDesc()));
- }
-
- return queryInfo;
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
deleted file mode 100644
index 641de78..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
+++ /dev/null
@@ -1,631 +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.tajo.master.querymaster;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.CompositeService;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.hadoop.yarn.event.Event;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.util.Clock;
-import org.apache.hadoop.yarn.util.SystemClock;
-import org.apache.tajo.*;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.planner.global.GlobalPlanner;
-import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.ha.HAServiceUtil;
-import org.apache.tajo.ipc.TajoMasterProtocol;
-import org.apache.tajo.ipc.TajoWorkerProtocol;
-import org.apache.tajo.master.event.QueryStartEvent;
-import org.apache.tajo.master.event.QueryStopEvent;
-import org.apache.tajo.rpc.CallFuture;
-import org.apache.tajo.rpc.NettyClientBase;
-import org.apache.tajo.rpc.NullCallback;
-import org.apache.tajo.rpc.RpcConnectionPool;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
-import org.apache.tajo.util.NetUtils;
-import org.apache.tajo.util.history.QueryHistory;
-import org.apache.tajo.worker.TajoWorker;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static org.apache.tajo.ipc.TajoMasterProtocol.TajoHeartbeat;
-import static org.apache.tajo.ipc.TajoMasterProtocol.TajoHeartbeatResponse;
-
-// TODO - when exception, send error status to QueryJobManager
-public class QueryMaster extends CompositeService implements EventHandler {
- private static final Log LOG = LogFactory.getLog(QueryMaster.class.getName());
-
- private int querySessionTimeout;
-
- private Clock clock;
-
- private AsyncDispatcher dispatcher;
-
- private GlobalPlanner globalPlanner;
-
- private TajoConf systemConf;
-
- private Map<QueryId, QueryMasterTask> queryMasterTasks = Maps.newConcurrentMap();
-
- private Map<QueryId, QueryMasterTask> finishedQueryMasterTasks = Maps.newConcurrentMap();
-
- private ClientSessionTimeoutCheckThread clientSessionTimeoutCheckThread;
-
- private AtomicBoolean queryMasterStop = new AtomicBoolean(false);
-
- private QueryMasterContext queryMasterContext;
-
- private QueryContext queryContext;
-
- private QueryHeartbeatThread queryHeartbeatThread;
-
- private FinishedQueryMasterTaskCleanThread finishedQueryMasterTaskCleanThread;
-
- private TajoWorker.WorkerContext workerContext;
-
- private RpcConnectionPool connPool;
-
- private ExecutorService eventExecutor;
-
- public QueryMaster(TajoWorker.WorkerContext workerContext) {
- super(QueryMaster.class.getName());
- this.workerContext = workerContext;
- }
-
- public void init(Configuration conf) {
- LOG.info("QueryMaster init");
- try {
- this.systemConf = (TajoConf)conf;
- this.connPool = RpcConnectionPool.getPool(systemConf);
-
- querySessionTimeout = systemConf.getIntVar(TajoConf.ConfVars.QUERY_SESSION_TIMEOUT);
- queryMasterContext = new QueryMasterContext(systemConf);
-
- clock = new SystemClock();
-
- this.dispatcher = new AsyncDispatcher();
- addIfService(dispatcher);
-
- globalPlanner = new GlobalPlanner(systemConf, workerContext);
-
- dispatcher.register(QueryStartEvent.EventType.class, new QueryStartEventHandler());
- dispatcher.register(QueryStopEvent.EventType.class, new QueryStopEventHandler());
-
- } catch (Throwable t) {
- LOG.error(t.getMessage(), t);
- throw new RuntimeException(t);
- }
- super.init(conf);
- }
-
- @Override
- public void start() {
- LOG.info("QueryMaster start");
-
- queryHeartbeatThread = new QueryHeartbeatThread();
- queryHeartbeatThread.start();
-
- clientSessionTimeoutCheckThread = new ClientSessionTimeoutCheckThread();
- clientSessionTimeoutCheckThread.start();
-
- finishedQueryMasterTaskCleanThread = new FinishedQueryMasterTaskCleanThread();
- finishedQueryMasterTaskCleanThread.start();
-
- eventExecutor = Executors.newSingleThreadExecutor();
- super.start();
- }
-
- @Override
- public void stop() {
- if(queryMasterStop.getAndSet(true)){
- return;
- }
-
- if(queryHeartbeatThread != null) {
- queryHeartbeatThread.interrupt();
- }
-
- if(clientSessionTimeoutCheckThread != null) {
- clientSessionTimeoutCheckThread.interrupt();
- }
-
- if(finishedQueryMasterTaskCleanThread != null) {
- finishedQueryMasterTaskCleanThread.interrupt();
- }
-
- if(eventExecutor != null){
- eventExecutor.shutdown();
- }
-
- super.stop();
-
- LOG.info("QueryMaster stop");
- if(queryMasterContext.getWorkerContext().isYarnContainerMode()) {
- queryMasterContext.getWorkerContext().stopWorker(true);
- }
- }
-
- protected void cleanupExecutionBlock(List<TajoIdProtos.ExecutionBlockIdProto> executionBlockIds) {
- StringBuilder cleanupMessage = new StringBuilder();
- String prefix = "";
- for (TajoIdProtos.ExecutionBlockIdProto eachEbId: executionBlockIds) {
- cleanupMessage.append(prefix).append(new ExecutionBlockId(eachEbId).toString());
- prefix = ",";
- }
- LOG.info("cleanup executionBlocks: " + cleanupMessage);
- NettyClientBase rpc = null;
- List<TajoMasterProtocol.WorkerResourceProto> workers = getAllWorker();
- TajoWorkerProtocol.ExecutionBlockListProto.Builder builder = TajoWorkerProtocol.ExecutionBlockListProto.newBuilder();
- builder.addAllExecutionBlockId(Lists.newArrayList(executionBlockIds));
- TajoWorkerProtocol.ExecutionBlockListProto executionBlockListProto = builder.build();
-
- for (TajoMasterProtocol.WorkerResourceProto worker : workers) {
- try {
- TajoProtos.WorkerConnectionInfoProto connectionInfo = worker.getConnectionInfo();
- rpc = connPool.getConnection(NetUtils.createSocketAddr(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()),
- TajoWorkerProtocol.class, true);
- TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerProtocolService = rpc.getStub();
-
- tajoWorkerProtocolService.cleanupExecutionBlocks(null, executionBlockListProto, NullCallback.get());
- } catch (Exception e) {
- continue;
- } finally {
- connPool.releaseConnection(rpc);
- }
- }
- }
-
- private void cleanup(QueryId queryId) {
- LOG.info("cleanup query resources : " + queryId);
- NettyClientBase rpc = null;
- List<TajoMasterProtocol.WorkerResourceProto> workers = getAllWorker();
-
- for (TajoMasterProtocol.WorkerResourceProto worker : workers) {
- try {
- TajoProtos.WorkerConnectionInfoProto connectionInfo = worker.getConnectionInfo();
- rpc = connPool.getConnection(NetUtils.createSocketAddr(connectionInfo.getHost(), connectionInfo.getPeerRpcPort()),
- TajoWorkerProtocol.class, true);
- TajoWorkerProtocol.TajoWorkerProtocolService tajoWorkerProtocolService = rpc.getStub();
-
- tajoWorkerProtocolService.cleanup(null, queryId.getProto(), NullCallback.get());
- } catch (Exception e) {
- LOG.error(e.getMessage());
- } finally {
- connPool.releaseConnection(rpc);
- }
- }
- }
-
- public List<TajoMasterProtocol.WorkerResourceProto> getAllWorker() {
-
- NettyClientBase rpc = null;
- try {
- // In TajoMaster HA mode, if backup master be active status,
- // worker may fail to connect existing active master. Thus,
- // if worker can't connect the master, worker should try to connect another master and
- // update master address in worker context.
- if (systemConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
- try {
- rpc = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- } catch (Exception e) {
- queryMasterContext.getWorkerContext().setWorkerResourceTrackerAddr(
- HAServiceUtil.getResourceTrackerAddress(systemConf));
- queryMasterContext.getWorkerContext().setTajoMasterAddress(
- HAServiceUtil.getMasterUmbilicalAddress(systemConf));
- rpc = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- }
- } else {
- rpc = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- }
-
- TajoMasterProtocol.TajoMasterProtocolService masterService = rpc.getStub();
-
- CallFuture<TajoMasterProtocol.WorkerResourcesRequest> callBack =
- new CallFuture<TajoMasterProtocol.WorkerResourcesRequest>();
- masterService.getAllWorkerResource(callBack.getController(),
- PrimitiveProtos.NullProto.getDefaultInstance(), callBack);
-
- TajoMasterProtocol.WorkerResourcesRequest workerResourcesRequest = callBack.get(2, TimeUnit.SECONDS);
- return workerResourcesRequest.getWorkerResourcesList();
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- } finally {
- connPool.releaseConnection(rpc);
- }
- return new ArrayList<TajoMasterProtocol.WorkerResourceProto>();
- }
-
- public void reportQueryStatusToQueryMaster(QueryId queryId, TajoProtos.QueryState state) {
- LOG.info("Send QueryMaster Ready to QueryJobManager:" + queryId);
- NettyClientBase tmClient = null;
- try {
- // In TajoMaster HA mode, if backup master be active status,
- // worker may fail to connect existing active master. Thus,
- // if worker can't connect the master, worker should try to connect another master and
- // update master address in worker context.
- if (systemConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
- try {
- tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- } catch (Exception e) {
- queryMasterContext.getWorkerContext().setWorkerResourceTrackerAddr(
- HAServiceUtil.getResourceTrackerAddress(systemConf));
- queryMasterContext.getWorkerContext().setTajoMasterAddress(
- HAServiceUtil.getMasterUmbilicalAddress(systemConf));
- tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- }
- } else {
- tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- }
-
- TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
-
- TajoHeartbeat.Builder queryHeartbeatBuilder = TajoHeartbeat.newBuilder()
- .setConnectionInfo(workerContext.getConnectionInfo().getProto())
- .setState(state)
- .setQueryId(queryId.getProto());
-
- CallFuture<TajoHeartbeatResponse> callBack =
- new CallFuture<TajoHeartbeatResponse>();
-
- masterClientService.heartbeat(callBack.getController(), queryHeartbeatBuilder.build(), callBack);
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- } finally {
- connPool.releaseConnection(tmClient);
- }
- }
-
- @Override
- public void handle(Event event) {
- dispatcher.getEventHandler().handle(event);
- }
-
- public Query getQuery(QueryId queryId) {
- return queryMasterTasks.get(queryId).getQuery();
- }
-
- public QueryMasterTask getQueryMasterTask(QueryId queryId) {
- return queryMasterTasks.get(queryId);
- }
-
- public QueryMasterTask getQueryMasterTask(QueryId queryId, boolean includeFinished) {
- QueryMasterTask queryMasterTask = queryMasterTasks.get(queryId);
- if(queryMasterTask != null) {
- return queryMasterTask;
- } else {
- if(includeFinished) {
- return finishedQueryMasterTasks.get(queryId);
- } else {
- return null;
- }
- }
- }
-
- public QueryMasterContext getContext() {
- return this.queryMasterContext;
- }
-
- public Collection<QueryMasterTask> getQueryMasterTasks() {
- return queryMasterTasks.values();
- }
-
- public Collection<QueryMasterTask> getFinishedQueryMasterTasks() {
- return finishedQueryMasterTasks.values();
- }
-
- public class QueryMasterContext {
- private TajoConf conf;
-
- public QueryMasterContext(TajoConf conf) {
- this.conf = conf;
- }
-
- public TajoConf getConf() {
- return conf;
- }
-
- public ExecutorService getEventExecutor(){
- return eventExecutor;
- }
-
- public AsyncDispatcher getDispatcher() {
- return dispatcher;
- }
-
- public Clock getClock() {
- return clock;
- }
-
- public QueryMaster getQueryMaster() {
- return QueryMaster.this;
- }
-
- public GlobalPlanner getGlobalPlanner() {
- return globalPlanner;
- }
-
- public TajoWorker.WorkerContext getWorkerContext() {
- return workerContext;
- }
-
- public EventHandler getEventHandler() {
- return dispatcher.getEventHandler();
- }
-
- public void stopQuery(QueryId queryId) {
- QueryMasterTask queryMasterTask = queryMasterTasks.remove(queryId);
- if(queryMasterTask == null) {
- LOG.warn("No query info:" + queryId);
- return;
- }
-
- finishedQueryMasterTasks.put(queryId, queryMasterTask);
-
- TajoHeartbeat queryHeartbeat = buildTajoHeartBeat(queryMasterTask);
- CallFuture<TajoHeartbeatResponse> future = new CallFuture<TajoHeartbeatResponse>();
-
- NettyClientBase tmClient = null;
- try {
- // In TajoMaster HA mode, if backup master be active status,
- // worker may fail to connect existing active master. Thus,
- // if worker can't connect the master, worker should try to connect another master and
- // update master address in worker context.
- if (systemConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
- try {
- tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- } catch (Exception e) {
- queryMasterContext.getWorkerContext().setWorkerResourceTrackerAddr(HAServiceUtil.getResourceTrackerAddress(systemConf));
- queryMasterContext.getWorkerContext().setTajoMasterAddress(HAServiceUtil.getMasterUmbilicalAddress(systemConf));
- tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- }
- } else {
- tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- }
-
- TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
- masterClientService.heartbeat(future.getController(), queryHeartbeat, future);
- } catch (Exception e) {
- //this function will be closed in new thread.
- //When tajo do stop cluster, tajo master maybe throw closed connection exception
-
- LOG.error(e.getMessage(), e);
- } finally {
- connPool.releaseConnection(tmClient);
- }
-
- try {
- queryMasterTask.stop();
- if (!queryContext.getBool(SessionVars.DEBUG_ENABLED)) {
- cleanup(queryId);
- }
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- }
- Query query = queryMasterTask.getQuery();
- if (query != null) {
- QueryHistory queryHisory = query.getQueryHistory();
- if (queryHisory != null) {
- query.context.getQueryMasterContext().getWorkerContext().
- getTaskHistoryWriter().appendHistory(queryHisory);
- }
- }
- if(workerContext.isYarnContainerMode()) {
- stop();
- }
- }
- }
-
- private TajoHeartbeat buildTajoHeartBeat(QueryMasterTask queryMasterTask) {
- TajoHeartbeat.Builder builder = TajoHeartbeat.newBuilder();
-
- builder.setConnectionInfo(workerContext.getConnectionInfo().getProto());
- builder.setQueryId(queryMasterTask.getQueryId().getProto());
- builder.setState(queryMasterTask.getState());
- if (queryMasterTask.getQuery() != null) {
- if (queryMasterTask.getQuery().getResultDesc() != null) {
- builder.setResultDesc(queryMasterTask.getQuery().getResultDesc().getProto());
- }
- builder.setQueryProgress(queryMasterTask.getQuery().getProgress());
- builder.setQueryFinishTime(queryMasterTask.getQuery().getFinishTime());
- }
- return builder.build();
- }
-
- private class QueryStartEventHandler implements EventHandler<QueryStartEvent> {
- @Override
- public void handle(QueryStartEvent event) {
- LOG.info("Start QueryStartEventHandler:" + event.getQueryId());
- QueryMasterTask queryMasterTask = new QueryMasterTask(queryMasterContext,
- event.getQueryId(), event.getSession(), event.getQueryContext(), event.getJsonExpr(), event.getLogicalPlanJson());
-
- synchronized(queryMasterTasks) {
- queryMasterTasks.put(event.getQueryId(), queryMasterTask);
- }
-
- queryMasterTask.init(systemConf);
- if (!queryMasterTask.isInitError()) {
- queryMasterTask.start();
- }
-
- queryContext = event.getQueryContext();
-
- if (queryMasterTask.isInitError()) {
- queryMasterContext.stopQuery(queryMasterTask.getQueryId());
- return;
- }
- }
- }
-
- private class QueryStopEventHandler implements EventHandler<QueryStopEvent> {
- @Override
- public void handle(QueryStopEvent event) {
- queryMasterContext.stopQuery(event.getQueryId());
- }
- }
-
- class QueryHeartbeatThread extends Thread {
- public QueryHeartbeatThread() {
- super("QueryHeartbeatThread");
- }
-
- @Override
- public void run() {
- LOG.info("Start QueryMaster heartbeat thread");
- while(!queryMasterStop.get()) {
- List<QueryMasterTask> tempTasks = new ArrayList<QueryMasterTask>();
- synchronized(queryMasterTasks) {
- tempTasks.addAll(queryMasterTasks.values());
- }
- synchronized(queryMasterTasks) {
- for(QueryMasterTask eachTask: tempTasks) {
- NettyClientBase tmClient;
- try {
- // In TajoMaster HA mode, if backup master be active status,
- // worker may fail to connect existing active master. Thus,
- // if worker can't connect the master, worker should try to connect another master and
- // update master address in worker context.
- if (systemConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
- try {
- tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- } catch (Exception e) {
- queryMasterContext.getWorkerContext().setWorkerResourceTrackerAddr(
- HAServiceUtil.getResourceTrackerAddress(systemConf));
- queryMasterContext.getWorkerContext().setTajoMasterAddress(
- HAServiceUtil.getMasterUmbilicalAddress(systemConf));
- tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- }
- } else {
- tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- }
-
- TajoMasterProtocol.TajoMasterProtocolService masterClientService = tmClient.getStub();
-
- CallFuture<TajoHeartbeatResponse> callBack =
- new CallFuture<TajoHeartbeatResponse>();
-
- TajoHeartbeat queryHeartbeat = buildTajoHeartBeat(eachTask);
- masterClientService.heartbeat(callBack.getController(), queryHeartbeat, callBack);
- } catch (Throwable t) {
- t.printStackTrace();
- }
- }
- }
- synchronized(queryMasterStop) {
- try {
- queryMasterStop.wait(2000);
- } catch (InterruptedException e) {
- break;
- }
- }
- }
- LOG.info("QueryMaster heartbeat thread stopped");
- }
- }
-
- class ClientSessionTimeoutCheckThread extends Thread {
- public void run() {
- LOG.info("ClientSessionTimeoutCheckThread started");
- while(!queryMasterStop.get()) {
- try {
- Thread.sleep(1000);
- } catch (InterruptedException e) {
- break;
- }
- List<QueryMasterTask> tempTasks = new ArrayList<QueryMasterTask>();
- synchronized(queryMasterTasks) {
- tempTasks.addAll(queryMasterTasks.values());
- }
-
- for(QueryMasterTask eachTask: tempTasks) {
- if(!eachTask.isStopped()) {
- try {
- long lastHeartbeat = eachTask.getLastClientHeartbeat();
- long time = System.currentTimeMillis() - lastHeartbeat;
- if(lastHeartbeat > 0 && time > querySessionTimeout * 1000) {
- LOG.warn("Query " + eachTask.getQueryId() + " stopped cause query session timeout: " + time + " ms");
- eachTask.expireQuerySession();
- }
- } catch (Exception e) {
- LOG.error(eachTask.getQueryId() + ":" + e.getMessage(), e);
- }
- }
- }
- }
- }
- }
-
- class FinishedQueryMasterTaskCleanThread extends Thread {
- public void run() {
- int expireIntervalTime = systemConf.getIntVar(TajoConf.ConfVars.WORKER_HISTORY_EXPIRE_PERIOD);
- LOG.info("FinishedQueryMasterTaskCleanThread started: expire interval minutes = " + expireIntervalTime);
- while(!queryMasterStop.get()) {
- try {
- Thread.sleep(60 * 1000 * 60); // hourly
- } catch (InterruptedException e) {
- break;
- }
- try {
- long expireTime = System.currentTimeMillis() - expireIntervalTime * 60 * 1000;
- cleanExpiredFinishedQueryMasterTask(expireTime);
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- }
- }
- }
-
- private void cleanExpiredFinishedQueryMasterTask(long expireTime) {
- synchronized(finishedQueryMasterTasks) {
- List<QueryId> expiredQueryIds = new ArrayList<QueryId>();
- for(Map.Entry<QueryId, QueryMasterTask> entry: finishedQueryMasterTasks.entrySet()) {
- if(entry.getValue().getStartTime() < expireTime) {
- expiredQueryIds.add(entry.getKey());
- }
- }
-
- for(QueryId eachId: expiredQueryIds) {
- finishedQueryMasterTasks.remove(eachId);
- }
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
deleted file mode 100644
index 9f7d3f8..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java
+++ /dev/null
@@ -1,263 +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.tajo.master.querymaster;
-
-import com.google.common.base.Preconditions;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.service.CompositeService;
-import org.apache.tajo.*;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.ipc.QueryMasterProtocol;
-import org.apache.tajo.ipc.TajoWorkerProtocol;
-import org.apache.tajo.master.DefaultTaskScheduler;
-import org.apache.tajo.master.container.TajoContainerId;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.session.Session;
-import org.apache.tajo.rpc.AsyncRpcServer;
-import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
-import org.apache.tajo.util.NetUtils;
-import org.apache.tajo.worker.TajoWorker;
-
-import java.net.InetSocketAddress;
-
-public class QueryMasterManagerService extends CompositeService
- implements QueryMasterProtocol.QueryMasterProtocolService.Interface {
- private static final Log LOG = LogFactory.getLog(QueryMasterManagerService.class.getName());
-
- private AsyncRpcServer rpcServer;
- private InetSocketAddress bindAddr;
- private String addr;
- private int port;
-
- private QueryMaster queryMaster;
-
- private TajoWorker.WorkerContext workerContext;
-
- public QueryMasterManagerService(TajoWorker.WorkerContext workerContext, int port) {
- super(QueryMasterManagerService.class.getName());
- this.workerContext = workerContext;
- this.port = port;
- }
-
- public QueryMaster getQueryMaster() {
- return queryMaster;
- }
-
- @Override
- public void init(Configuration conf) {
- Preconditions.checkArgument(conf instanceof TajoConf);
- TajoConf tajoConf = (TajoConf) conf;
- try {
- // Setup RPC server
- InetSocketAddress initIsa =
- new InetSocketAddress("0.0.0.0", port);
- if (initIsa.getAddress() == null) {
- throw new IllegalArgumentException("Failed resolve of " + initIsa);
- }
-
- int workerNum = tajoConf.getIntVar(TajoConf.ConfVars.QUERY_MASTER_RPC_SERVER_WORKER_THREAD_NUM);
- this.rpcServer = new AsyncRpcServer(QueryMasterProtocol.class, this, initIsa, workerNum);
- this.rpcServer.start();
-
- this.bindAddr = NetUtils.getConnectAddress(rpcServer.getListenAddress());
- this.addr = bindAddr.getHostName() + ":" + bindAddr.getPort();
-
- this.port = bindAddr.getPort();
-
- queryMaster = new QueryMaster(workerContext);
- addService(queryMaster);
-
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- }
- // Get the master address
- LOG.info("QueryMasterManagerService is bind to " + addr);
- ((TajoConf)conf).setVar(TajoConf.ConfVars.WORKER_QM_RPC_ADDRESS, addr);
-
- super.init(conf);
- }
-
- @Override
- public void start() {
- super.start();
- }
-
- @Override
- public void stop() {
- if(rpcServer != null) {
- rpcServer.shutdown();
- }
- LOG.info("QueryMasterManagerService stopped");
- super.stop();
- }
-
- public InetSocketAddress getBindAddr() {
- return bindAddr;
- }
-
- public String getHostAndPort() {
- return bindAddr.getHostName() + ":" + bindAddr.getPort();
- }
-
- @Override
- public void getTask(RpcController controller, TajoWorkerProtocol.GetTaskRequestProto request,
- RpcCallback<TajoWorkerProtocol.TaskRequestProto> done) {
- try {
- ExecutionBlockId ebId = new ExecutionBlockId(request.getExecutionBlockId());
- QueryMasterTask queryMasterTask = workerContext.getQueryMaster().getQueryMasterTask(ebId.getQueryId());
-
- if(queryMasterTask == null || queryMasterTask.isStopped()) {
- done.run(DefaultTaskScheduler.stopTaskRunnerReq);
- } else {
- TajoContainerId cid =
- queryMasterTask.getQueryTaskContext().getResourceAllocator().makeContainerId(request.getContainerId());
- LOG.debug("getTask:" + cid + ", ebId:" + ebId);
- queryMasterTask.handleTaskRequestEvent(new TaskRequestEvent(request.getWorkerId(), cid, ebId, done));
- }
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- }
- }
-
- @Override
- public void statusUpdate(RpcController controller, TajoWorkerProtocol.TaskStatusProto request,
- RpcCallback<PrimitiveProtos.BoolProto> done) {
- try {
- QueryId queryId = new QueryId(request.getId().getTaskId().getExecutionBlockId().getQueryId());
- TaskAttemptId attemptId = new TaskAttemptId(request.getId());
- QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(queryId);
- if (queryMasterTask == null) {
- queryMasterTask = queryMaster.getQueryMasterTask(queryId, true);
- }
- Stage sq = queryMasterTask.getQuery().getStage(attemptId.getTaskId().getExecutionBlockId());
- Task task = sq.getTask(attemptId.getTaskId());
- TaskAttempt attempt = task.getAttempt(attemptId.getId());
-
- if(LOG.isDebugEnabled()){
- LOG.debug(String.format("Task State: %s, Attempt State: %s", task.getState().name(), attempt.getState().name()));
- }
-
- if (request.getState() == TajoProtos.TaskAttemptState.TA_KILLED) {
- LOG.warn(attemptId + " Killed");
- attempt.handle(
- new TaskAttemptEvent(new TaskAttemptId(request.getId()), TaskAttemptEventType.TA_LOCAL_KILLED));
- } else {
- queryMasterTask.getEventHandler().handle(
- new TaskAttemptStatusUpdateEvent(new TaskAttemptId(request.getId()), request));
- }
- done.run(TajoWorker.TRUE_PROTO);
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- done.run(TajoWorker.FALSE_PROTO);
- }
- }
-
- @Override
- public void ping(RpcController controller,
- TajoIdProtos.ExecutionBlockIdProto requestProto,
- RpcCallback<PrimitiveProtos.BoolProto> done) {
- done.run(TajoWorker.TRUE_PROTO);
- }
-
- @Override
- public void fatalError(RpcController controller, TajoWorkerProtocol.TaskFatalErrorReport report,
- RpcCallback<PrimitiveProtos.BoolProto> done) {
- try {
- QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(
- new QueryId(report.getId().getTaskId().getExecutionBlockId().getQueryId()));
- if (queryMasterTask != null) {
- queryMasterTask.handleTaskFailed(report);
- } else {
- LOG.warn("No QueryMasterTask: " + new TaskAttemptId(report.getId()));
- }
- done.run(TajoWorker.TRUE_PROTO);
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- done.run(TajoWorker.FALSE_PROTO);
- }
- }
-
- @Override
- public void done(RpcController controller, TajoWorkerProtocol.TaskCompletionReport report,
- RpcCallback<PrimitiveProtos.BoolProto> done) {
- try {
- QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(
- new QueryId(report.getId().getTaskId().getExecutionBlockId().getQueryId()));
- if (queryMasterTask != null) {
- queryMasterTask.getEventHandler().handle(new TaskCompletionEvent(report));
- }
- done.run(TajoWorker.TRUE_PROTO);
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- done.run(TajoWorker.FALSE_PROTO);
- }
- }
-
- @Override
- public void doneExecutionBlock(
- RpcController controller, TajoWorkerProtocol.ExecutionBlockReport request,
- RpcCallback<PrimitiveProtos.BoolProto> done) {
- QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(new QueryId(request.getEbId().getQueryId()));
- if (queryMasterTask != null) {
- ExecutionBlockId ebId = new ExecutionBlockId(request.getEbId());
- queryMasterTask.getQuery().getStage(ebId).receiveExecutionBlockReport(request);
- }
- done.run(TajoWorker.TRUE_PROTO);
- }
-
- @Override
- public void killQuery(RpcController controller, TajoIdProtos.QueryIdProto request,
- RpcCallback<PrimitiveProtos.BoolProto> done) {
- QueryId queryId = new QueryId(request);
- QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(queryId);
- if (queryMasterTask != null) {
- Query query = queryMasterTask.getQuery();
- if (query != null) {
- query.handle(new QueryEvent(queryId, QueryEventType.KILL));
- }
- }
- }
-
- @Override
- public void executeQuery(RpcController controller,
- TajoWorkerProtocol.QueryExecutionRequestProto request,
- RpcCallback<PrimitiveProtos.BoolProto> done) {
- try {
- workerContext.getWorkerSystemMetrics().counter("querymaster", "numQuery").inc();
-
- QueryId queryId = new QueryId(request.getQueryId());
- LOG.info("Receive executeQuery request:" + queryId);
- queryMaster.handle(new QueryStartEvent(queryId,
- new Session(request.getSession()),
- new QueryContext(workerContext.getQueryMaster().getContext().getConf(),
- request.getQueryContext()), request.getExprInJson().getValue(),
- request.getLogicalPlanJson().getValue()));
- done.run(TajoWorker.TRUE_PROTO);
- } catch (Exception e) {
- workerContext.getWorkerSystemMetrics().counter("querymaster", "errorQuery").inc();
- LOG.error(e.getMessage(), e);
- done.run(TajoWorker.FALSE_PROTO);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
deleted file mode 100644
index 56dd789..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterRunner.java
+++ /dev/null
@@ -1,149 +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.tajo.master.querymaster;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.util.RackResolver;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.TajoConstants;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.util.TajoIdUtils;
-
-import java.io.PrintWriter;
-import java.lang.management.ManagementFactory;
-import java.lang.management.ThreadInfo;
-import java.lang.management.ThreadMXBean;
-
-@Deprecated
-public class QueryMasterRunner extends AbstractService {
- private static final Log LOG = LogFactory.getLog(QueryMasterRunner.class);
- private TajoConf systemConf;
- private QueryMaster queryMaster;
- private QueryId queryId;
- private String queryMasterManagerAddress;
-
- public QueryMasterRunner(QueryId queryId, String queryMasterManagerAddress) {
- super(QueryMasterRunner.class.getName());
- this.queryId = queryId;
- this.queryMasterManagerAddress = queryMasterManagerAddress;
- }
-
- private class ShutdownHook implements Runnable {
- @Override
- public void run() {
- LOG.info("============================================");
- LOG.info("QueryMaster received SIGINT Signal");
- LOG.info("============================================");
- stop();
- }
- }
-
- @Override
- public void init(Configuration conf) {
- this.systemConf = (TajoConf)conf;
- RackResolver.init(systemConf);
- Runtime.getRuntime().addShutdownHook(new Thread(new ShutdownHook()));
- super.init(conf);
- }
-
- @Override
- public void start() {
- //create QueryMaster
- QueryMaster query = new QueryMaster(null);
-
- query.init(systemConf);
- query.start();
- }
-
- @Override
- public void stop() {
- }
-
- public static void main(String[] args) throws Exception {
- LOG.info("QueryMasterRunner started");
-
- final TajoConf conf = new TajoConf();
- conf.addResource(new Path(TajoConstants.SYSTEM_CONF_FILENAME));
-
- UserGroupInformation.setConfiguration(conf);
-
- final QueryId queryId = TajoIdUtils.parseQueryId(args[0]);
- final String queryMasterManagerAddr = args[1];
-
- LOG.info("Received QueryId:" + queryId);
-
- QueryMasterRunner queryMasterRunner = new QueryMasterRunner(queryId, queryMasterManagerAddr);
- queryMasterRunner.init(conf);
- queryMasterRunner.start();
-
- synchronized(queryId) {
- queryId.wait();
- }
-
- System.exit(0);
- }
-
- public static void printThreadInfo(PrintWriter stream, String title) {
- ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
- final int STACK_DEPTH = 60;
- boolean contention = threadBean.isThreadContentionMonitoringEnabled();
- long[] threadIds = threadBean.getAllThreadIds();
- stream.println("Process Thread Dump: " + title);
- stream.println(threadIds.length + " active threads");
- for (long tid : threadIds) {
- ThreadInfo info = threadBean.getThreadInfo(tid, STACK_DEPTH);
- if (info == null) {
- stream.println(" Inactive");
- continue;
- }
- stream.println("Thread " + getTaskName(info.getThreadId(), info.getThreadName()) + ":");
- Thread.State state = info.getThreadState();
- stream.println(" State: " + state);
- stream.println(" Blocked count: " + info.getBlockedCount());
- stream.println(" Waited count: " + info.getWaitedCount());
- if (contention) {
- stream.println(" Blocked time: " + info.getBlockedTime());
- stream.println(" Waited time: " + info.getWaitedTime());
- }
- if (state == Thread.State.WAITING) {
- stream.println(" Waiting on " + info.getLockName());
- } else if (state == Thread.State.BLOCKED) {
- stream.println(" Blocked on " + info.getLockName());
- stream.println(" Blocked by " + getTaskName(info.getLockOwnerId(), info.getLockOwnerName()));
- }
- stream.println(" Stack:");
- for (StackTraceElement frame : info.getStackTrace()) {
- stream.println(" " + frame.toString());
- }
- }
- stream.flush();
- }
-
- private static String getTaskName(long id, String name) {
- if (name == null) {
- return Long.toString(id);
- }
- return id + " (" + name + ")";
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
deleted file mode 100644
index 9c789a5..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
+++ /dev/null
@@ -1,638 +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.tajo.master.querymaster;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.service.CompositeService;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.util.Clock;
-import org.apache.tajo.*;
-import org.apache.tajo.algebra.Expr;
-import org.apache.tajo.algebra.JsonHelper;
-import org.apache.tajo.catalog.CatalogService;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.exception.UnimplementedException;
-import org.apache.tajo.ha.HAServiceUtil;
-import org.apache.tajo.ipc.TajoMasterProtocol;
-import org.apache.tajo.ipc.TajoWorkerProtocol;
-import org.apache.tajo.master.TajoContainerProxy;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.rm.TajoWorkerResourceManager;
-import org.apache.tajo.master.session.Session;
-import org.apache.tajo.plan.LogicalOptimizer;
-import org.apache.tajo.plan.LogicalPlan;
-import org.apache.tajo.plan.LogicalPlanner;
-import org.apache.tajo.plan.logical.LogicalNode;
-import org.apache.tajo.plan.logical.LogicalRootNode;
-import org.apache.tajo.plan.logical.NodeType;
-import org.apache.tajo.plan.logical.ScanNode;
-import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRule;
-import org.apache.tajo.plan.util.PlannerUtil;
-import org.apache.tajo.plan.verifier.VerifyException;
-import org.apache.tajo.rpc.NettyClientBase;
-import org.apache.tajo.rpc.RpcConnectionPool;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.StorageProperty;
-import org.apache.tajo.storage.StorageUtil;
-import org.apache.tajo.util.metrics.TajoMetrics;
-import org.apache.tajo.util.metrics.reporter.MetricsConsoleReporter;
-import org.apache.tajo.worker.AbstractResourceAllocator;
-import org.apache.tajo.worker.TajoResourceAllocator;
-
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static org.apache.tajo.TajoProtos.QueryState;
-
-public class QueryMasterTask extends CompositeService {
- private static final Log LOG = LogFactory.getLog(QueryMasterTask.class.getName());
-
- // query submission directory is private!
- final public static FsPermission STAGING_DIR_PERMISSION =
- FsPermission.createImmutable((short) 0700); // rwx--------
-
- public static final String TMP_STAGING_DIR_PREFIX = ".staging";
-
- private QueryId queryId;
-
- private Session session;
-
- private QueryContext queryContext;
-
- private QueryMasterTaskContext queryTaskContext;
-
- private QueryMaster.QueryMasterContext queryMasterContext;
-
- private Query query;
-
- private MasterPlan masterPlan;
-
- private String jsonExpr;
-
- private String logicalPlanJson;
-
- private AsyncDispatcher dispatcher;
-
- private final long querySubmitTime;
-
- private Map<String, TableDesc> tableDescMap = new HashMap<String, TableDesc>();
-
- private TajoConf systemConf;
-
- private AtomicLong lastClientHeartbeat = new AtomicLong(-1);
-
- private AbstractResourceAllocator resourceAllocator;
-
- private AtomicBoolean stopped = new AtomicBoolean(false);
-
- private TajoMetrics queryMetrics;
-
- private Throwable initError;
-
- private final List<TajoWorkerProtocol.TaskFatalErrorReport> diagnostics =
- new ArrayList<TajoWorkerProtocol.TaskFatalErrorReport>();
-
- public QueryMasterTask(QueryMaster.QueryMasterContext queryMasterContext,
- QueryId queryId, Session session, QueryContext queryContext, String jsonExpr,
- String logicalPlanJson) {
-
- super(QueryMasterTask.class.getName());
- this.queryMasterContext = queryMasterContext;
- this.queryId = queryId;
- this.session = session;
- this.queryContext = queryContext;
- this.jsonExpr = jsonExpr;
- this.logicalPlanJson = logicalPlanJson;
- this.querySubmitTime = System.currentTimeMillis();
- }
-
- @Override
- public void init(Configuration conf) {
- systemConf = (TajoConf)conf;
-
- try {
- queryTaskContext = new QueryMasterTaskContext();
- String resourceManagerClassName = systemConf.getVar(TajoConf.ConfVars.RESOURCE_MANAGER_CLASS);
-
- if(resourceManagerClassName.indexOf(TajoWorkerResourceManager.class.getName()) >= 0) {
- resourceAllocator = new TajoResourceAllocator(queryTaskContext);
- } else {
- throw new UnimplementedException(resourceManagerClassName + " is not supported yet");
- }
- addService(resourceAllocator);
-
- dispatcher = new AsyncDispatcher();
- addService(dispatcher);
-
- dispatcher.register(StageEventType.class, new StageEventDispatcher());
- dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
- dispatcher.register(TaskAttemptEventType.class, new TaskAttemptEventDispatcher());
- dispatcher.register(QueryMasterQueryCompletedEvent.EventType.class, new QueryFinishEventHandler());
- dispatcher.register(TaskSchedulerEvent.EventType.class, new TaskSchedulerDispatcher());
- dispatcher.register(LocalTaskEventType.class, new LocalTaskEventHandler());
-
- initStagingDir();
-
- queryMetrics = new TajoMetrics(queryId.toString());
-
- super.init(systemConf);
- } catch (Throwable t) {
- LOG.error(t.getMessage(), t);
- initError = t;
- }
- }
-
- public boolean isStopped() {
- return stopped.get();
- }
-
- @Override
- public void start() {
- startQuery();
- super.start();
- }
-
- @Override
- public void stop() {
-
- if(stopped.getAndSet(true)) {
- return;
- }
-
- LOG.info("Stopping QueryMasterTask:" + queryId);
-
- try {
- resourceAllocator.stop();
- } catch (Throwable t) {
- LOG.fatal(t.getMessage(), t);
- }
-
- RpcConnectionPool connPool = RpcConnectionPool.getPool(queryMasterContext.getConf());
- NettyClientBase tmClient = null;
- try {
- // In TajoMaster HA mode, if backup master be active status,
- // worker may fail to connect existing active master. Thus,
- // if worker can't connect the master, worker should try to connect another master and
- // update master address in worker context.
- if (systemConf.getBoolVar(TajoConf.ConfVars.TAJO_MASTER_HA_ENABLE)) {
- try {
- tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- } catch (Exception e) {
- queryMasterContext.getWorkerContext().setWorkerResourceTrackerAddr(
- HAServiceUtil.getResourceTrackerAddress(systemConf));
- queryMasterContext.getWorkerContext().setTajoMasterAddress(
- HAServiceUtil.getMasterUmbilicalAddress(systemConf));
- tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- }
- } else {
- tmClient = connPool.getConnection(queryMasterContext.getWorkerContext().getTajoMasterAddress(),
- TajoMasterProtocol.class, true);
- }
- } catch (Exception e) {
- LOG.error(e.getMessage(), e);
- } finally {
- connPool.releaseConnection(tmClient);
- }
-
- super.stop();
-
- //TODO change report to tajo master
- if (queryMetrics != null) {
- queryMetrics.report(new MetricsConsoleReporter());
- }
-
- LOG.info("Stopped QueryMasterTask:" + queryId);
- }
-
- public void handleTaskRequestEvent(TaskRequestEvent event) {
- ExecutionBlockId id = event.getExecutionBlockId();
- query.getStage(id).handleTaskRequestEvent(event);
- }
-
- public void handleTaskFailed(TajoWorkerProtocol.TaskFatalErrorReport report) {
- synchronized(diagnostics) {
- if (diagnostics.size() < 10) {
- diagnostics.add(report);
- }
- }
-
- getEventHandler().handle(new TaskFatalErrorEvent(report));
- }
-
- public Collection<TajoWorkerProtocol.TaskFatalErrorReport> getDiagnostics() {
- synchronized(diagnostics) {
- return Collections.unmodifiableCollection(diagnostics);
- }
- }
-
- private class StageEventDispatcher implements EventHandler<StageEvent> {
- public void handle(StageEvent event) {
- ExecutionBlockId id = event.getStageId();
- if(LOG.isDebugEnabled()) {
- LOG.debug("StageEventDispatcher:" + id + "," + event.getType());
- }
- query.getStage(id).handle(event);
- }
- }
-
- private class TaskEventDispatcher
- implements EventHandler<TaskEvent> {
- public void handle(TaskEvent event) {
- TaskId taskId = event.getTaskId();
- if(LOG.isDebugEnabled()) {
- LOG.debug("TaskEventDispatcher>" + taskId + "," + event.getType());
- }
- Task task = query.getStage(taskId.getExecutionBlockId()).
- getTask(taskId);
- task.handle(event);
- }
- }
-
- private class TaskAttemptEventDispatcher
- implements EventHandler<TaskAttemptEvent> {
- public void handle(TaskAttemptEvent event) {
- TaskAttemptId attemptId = event.getTaskAttemptId();
- Stage stage = query.getStage(attemptId.getTaskId().getExecutionBlockId());
- Task task = stage.getTask(attemptId.getTaskId());
- TaskAttempt attempt = task.getAttempt(attemptId);
- attempt.handle(event);
- }
- }
-
- private class TaskSchedulerDispatcher
- implements EventHandler<TaskSchedulerEvent> {
- public void handle(TaskSchedulerEvent event) {
- Stage stage = query.getStage(event.getExecutionBlockId());
- stage.getTaskScheduler().handle(event);
- }
- }
-
- private class LocalTaskEventHandler implements EventHandler<LocalTaskEvent> {
- @Override
- public void handle(LocalTaskEvent event) {
- TajoContainerProxy proxy = (TajoContainerProxy) resourceAllocator.getContainers().get(event.getContainerId());
- if (proxy != null) {
- proxy.killTaskAttempt(event.getTaskAttemptId());
- }
- }
- }
-
- private class QueryFinishEventHandler implements EventHandler<QueryMasterQueryCompletedEvent> {
- @Override
- public void handle(QueryMasterQueryCompletedEvent event) {
- QueryId queryId = event.getQueryId();
- LOG.info("Query completion notified from " + queryId);
-
- while (!isTerminatedState(query.getSynchronizedState())) {
- try {
- synchronized (this) {
- wait(10);
- }
- } catch (InterruptedException e) {
- LOG.error(e);
- }
- }
- LOG.info("Query final state: " + query.getSynchronizedState());
-
- queryMasterContext.getEventHandler().handle(new QueryStopEvent(queryId));
- }
- }
-
- private static boolean isTerminatedState(QueryState state) {
- return
- state == QueryState.QUERY_SUCCEEDED ||
- state == QueryState.QUERY_FAILED ||
- state == QueryState.QUERY_KILLED ||
- state == QueryState.QUERY_ERROR;
- }
-
- public synchronized void startQuery() {
- StorageManager sm = null;
- LogicalPlan plan = null;
- try {
- if (query != null) {
- LOG.warn("Query already started");
- return;
- }
- CatalogService catalog = getQueryTaskContext().getQueryMasterContext().getWorkerContext().getCatalog();
- LogicalPlanner planner = new LogicalPlanner(catalog);
- LogicalOptimizer optimizer = new LogicalOptimizer(systemConf);
- Expr expr = JsonHelper.fromJson(jsonExpr, Expr.class);
- jsonExpr = null; // remove the possible OOM
- plan = planner.createPlan(queryContext, expr);
-
- StoreType storeType = PlannerUtil.getStoreType(plan);
- if (storeType != null) {
- sm = StorageManager.getStorageManager(systemConf, storeType);
- StorageProperty storageProperty = sm.getStorageProperty();
- if (storageProperty.isSortedInsert()) {
- String tableName = PlannerUtil.getStoreTableName(plan);
- LogicalRootNode rootNode = plan.getRootBlock().getRoot();
- TableDesc tableDesc = PlannerUtil.getTableDesc(catalog, rootNode.getChild());
- if (tableDesc == null) {
- throw new VerifyException("Can't get table meta data from catalog: " + tableName);
- }
- List<LogicalPlanRewriteRule> storageSpecifiedRewriteRules = sm.getRewriteRules(
- getQueryTaskContext().getQueryContext(), tableDesc);
- if (storageSpecifiedRewriteRules != null) {
- for (LogicalPlanRewriteRule eachRule: storageSpecifiedRewriteRules) {
- optimizer.addRuleAfterToJoinOpt(eachRule);
- }
- }
- }
- }
-
- optimizer.optimize(queryContext, plan);
-
- for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
- LogicalNode[] scanNodes = PlannerUtil.findAllNodes(block.getRoot(), NodeType.SCAN);
- if (scanNodes != null) {
- for (LogicalNode eachScanNode : scanNodes) {
- ScanNode scanNode = (ScanNode) eachScanNode;
- tableDescMap.put(scanNode.getCanonicalName(), scanNode.getTableDesc());
- }
- }
-
- scanNodes = PlannerUtil.findAllNodes(block.getRoot(), NodeType.PARTITIONS_SCAN);
- if (scanNodes != null) {
- for (LogicalNode eachScanNode : scanNodes) {
- ScanNode scanNode = (ScanNode) eachScanNode;
- tableDescMap.put(scanNode.getCanonicalName(), scanNode.getTableDesc());
- }
- }
- }
- MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
- queryMasterContext.getGlobalPlanner().build(masterPlan);
-
- query = new Query(queryTaskContext, queryId, querySubmitTime,
- "", queryTaskContext.getEventHandler(), masterPlan);
-
- dispatcher.register(QueryEventType.class, query);
- queryTaskContext.getEventHandler().handle(new QueryEvent(queryId, QueryEventType.START));
- } catch (Throwable t) {
- LOG.error(t.getMessage(), t);
- initError = t;
-
- if (plan != null && sm != null) {
- LogicalRootNode rootNode = plan.getRootBlock().getRoot();
- try {
- sm.rollbackOutputCommit(rootNode.getChild());
- } catch (IOException e) {
- LOG.warn(query.getId() + ", failed processing cleanup storage when query failed:" + e.getMessage(), e);
- }
- }
- }
- }
-
- private void initStagingDir() throws IOException {
- Path stagingDir = null;
- FileSystem defaultFS = TajoConf.getWarehouseDir(systemConf).getFileSystem(systemConf);
-
- try {
-
- stagingDir = initStagingDir(systemConf, queryId.toString(), queryContext);
-
- // Create a subdirectories
- LOG.info("The staging dir '" + stagingDir + "' is created.");
- queryContext.setStagingDir(stagingDir);
- } catch (IOException ioe) {
- if (stagingDir != null && defaultFS.exists(stagingDir)) {
- try {
- defaultFS.delete(stagingDir, true);
- LOG.info("The staging directory '" + stagingDir + "' is deleted");
- } catch (Exception e) {
- LOG.warn(e.getMessage());
- }
- }
-
- throw ioe;
- }
- }
-
- /**
- * It initializes the final output and staging directory and sets
- * them to variables.
- */
- public static Path initStagingDir(TajoConf conf, String queryId, QueryContext context) throws IOException {
-
- String realUser;
- String currentUser;
- UserGroupInformation ugi;
- ugi = UserGroupInformation.getLoginUser();
- realUser = ugi.getShortUserName();
- currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
-
- FileSystem fs;
- Path stagingDir;
-
- ////////////////////////////////////////////
- // Create Output Directory
- ////////////////////////////////////////////
-
- String outputPath = context.get(QueryVars.OUTPUT_TABLE_PATH, "");
- if (context.isCreateTable() || context.isInsert()) {
- if (outputPath == null || outputPath.isEmpty()) {
- // hbase
- stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
- } else {
- stagingDir = StorageUtil.concatPath(context.getOutputPath(), TMP_STAGING_DIR_PREFIX, queryId);
- }
- } else {
- stagingDir = new Path(TajoConf.getDefaultRootStagingDir(conf), queryId);
- }
-
- // initializ
- fs = stagingDir.getFileSystem(conf);
-
- if (fs.exists(stagingDir)) {
- throw new IOException("The staging directory '" + stagingDir + "' already exists");
- }
- fs.mkdirs(stagingDir, new FsPermission(STAGING_DIR_PERMISSION));
- FileStatus fsStatus = fs.getFileStatus(stagingDir);
- String owner = fsStatus.getOwner();
-
- if (!owner.isEmpty() && !(owner.equals(currentUser) || owner.equals(realUser))) {
- throw new IOException("The ownership on the user's query " +
- "directory " + stagingDir + " is not as expected. " +
- "It is owned by " + owner + ". The directory must " +
- "be owned by the submitter " + currentUser + " or " +
- "by " + realUser);
- }
-
- if (!fsStatus.getPermission().equals(STAGING_DIR_PERMISSION)) {
- LOG.info("Permissions on staging directory " + stagingDir + " are " +
- "incorrect: " + fsStatus.getPermission() + ". Fixing permissions " +
- "to correct value " + STAGING_DIR_PERMISSION);
- fs.setPermission(stagingDir, new FsPermission(STAGING_DIR_PERMISSION));
- }
-
- Path stagingResultDir = new Path(stagingDir, TajoConstants.RESULT_DIR_NAME);
- fs.mkdirs(stagingResultDir);
-
- return stagingDir;
- }
-
- public Query getQuery() {
- return query;
- }
-
- protected void expireQuerySession() {
- if(!isTerminatedState(query.getState()) && !(query.getState() == QueryState.QUERY_KILL_WAIT)){
- query.handle(new QueryEvent(queryId, QueryEventType.KILL));
- }
- }
-
- public QueryMasterTaskContext getQueryTaskContext() {
- return queryTaskContext;
- }
-
- public EventHandler getEventHandler() {
- return queryTaskContext.getEventHandler();
- }
-
- public void touchSessionTime() {
- this.lastClientHeartbeat.set(System.currentTimeMillis());
- }
-
- public long getLastClientHeartbeat() {
- return this.lastClientHeartbeat.get();
- }
-
- public QueryId getQueryId() {
- return queryId;
- }
-
- public boolean isInitError() {
- return initError != null;
- }
-
- public QueryState getState() {
- if(query == null) {
- if (isInitError()) {
- return QueryState.QUERY_ERROR;
- } else {
- return QueryState.QUERY_NOT_ASSIGNED;
- }
- } else {
- return query.getState();
- }
- }
-
- public Throwable getInitError() {
- return initError;
- }
-
- public String getErrorMessage() {
- if (isInitError()) {
- return StringUtils.stringifyException(initError);
- } else {
- return null;
- }
- }
-
- public long getQuerySubmitTime() {
- return this.querySubmitTime;
- }
-
- public class QueryMasterTaskContext {
- EventHandler eventHandler;
- public QueryMaster.QueryMasterContext getQueryMasterContext() {
- return queryMasterContext;
- }
-
- public Session getSession() {
- return session;
- }
-
- public QueryContext getQueryContext() {
- return queryContext;
- }
-
- public TajoConf getConf() {
- return systemConf;
- }
-
- public Clock getClock() {
- return queryMasterContext.getClock();
- }
-
- public Query getQuery() {
- return query;
- }
-
- public QueryId getQueryId() {
- return queryId;
- }
-
- public Path getStagingDir() {
- return queryContext.getStagingDir();
- }
-
- public synchronized EventHandler getEventHandler() {
- if(eventHandler == null) {
- eventHandler = dispatcher.getEventHandler();
- }
- return eventHandler;
- }
-
- public AsyncDispatcher getDispatcher() {
- return dispatcher;
- }
-
- public Stage getStage(ExecutionBlockId id) {
- return query.getStage(id);
- }
-
- public Map<String, TableDesc> getTableDescMap() {
- return tableDescMap;
- }
-
- public float getProgress() {
- if(query == null) {
- return 0.0f;
- }
- return query.getProgress();
- }
-
- public AbstractResourceAllocator getResourceAllocator() {
- return resourceAllocator;
- }
-
- public TajoMetrics getQueryMetrics() {
- return queryMetrics;
- }
- }
-}
\ No newline at end of file
[07/16] tajo git commit: TAJO-1288: Refactoring
org.apache.tajo.master package.
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java
new file mode 100644
index 0000000..351856f
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/DefaultTaskScheduler.java
@@ -0,0 +1,926 @@
+/*
+ * 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.tajo.querymaster;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.RackResolver;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.TaskAttemptId;
+import org.apache.tajo.engine.planner.global.ExecutionBlock;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.engine.query.TaskRequest;
+import org.apache.tajo.engine.query.TaskRequestImpl;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.ContainerProxy;
+import org.apache.tajo.master.cluster.WorkerConnectionInfo;
+import org.apache.tajo.master.container.TajoContainerId;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext;
+import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
+import org.apache.tajo.plan.serder.LogicalNodeSerializer;
+import org.apache.tajo.plan.serder.PlanProto;
+import org.apache.tajo.storage.DataLocation;
+import org.apache.tajo.storage.fragment.FileFragment;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.worker.FetchImpl;
+
+import java.util.*;
+import java.util.Map.Entry;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+
+public class DefaultTaskScheduler extends AbstractTaskScheduler {
+ private static final Log LOG = LogFactory.getLog(DefaultTaskScheduler.class);
+
+ private final TaskSchedulerContext context;
+ private Stage stage;
+
+ private Thread schedulingThread;
+ private AtomicBoolean stopEventHandling = new AtomicBoolean(false);
+
+ private ScheduledRequests scheduledRequests;
+ private TaskRequests taskRequests;
+
+ private int nextTaskId = 0;
+ private int scheduledObjectNum = 0;
+
+ public DefaultTaskScheduler(TaskSchedulerContext context, Stage stage) {
+ super(DefaultTaskScheduler.class.getName());
+ this.context = context;
+ this.stage = stage;
+ }
+
+ @Override
+ public void init(Configuration conf) {
+
+ scheduledRequests = new ScheduledRequests();
+ taskRequests = new TaskRequests();
+
+ super.init(conf);
+ }
+
+ @Override
+ public void start() {
+ LOG.info("Start TaskScheduler");
+
+ this.schedulingThread = new Thread() {
+ public void run() {
+
+ while(!stopEventHandling.get() && !Thread.currentThread().isInterrupted()) {
+ try {
+ synchronized (schedulingThread){
+ schedulingThread.wait(100);
+ }
+ schedule();
+ } catch (InterruptedException e) {
+ break;
+ } catch (Throwable e) {
+ LOG.fatal(e.getMessage(), e);
+ break;
+ }
+ }
+ LOG.info("TaskScheduler schedulingThread stopped");
+ }
+ };
+
+ this.schedulingThread.start();
+ super.start();
+ }
+
+ private static final TaskAttemptId NULL_ATTEMPT_ID;
+ public static final TajoWorkerProtocol.TaskRequestProto stopTaskRunnerReq;
+ static {
+ ExecutionBlockId nullStage = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0);
+ NULL_ATTEMPT_ID = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(nullStage, 0), 0);
+
+ TajoWorkerProtocol.TaskRequestProto.Builder builder =
+ TajoWorkerProtocol.TaskRequestProto.newBuilder();
+ builder.setId(NULL_ATTEMPT_ID.getProto());
+ builder.setShouldDie(true);
+ builder.setOutputTable("");
+ builder.setPlan(PlanProto.LogicalNodeTree.newBuilder());
+ builder.setClusteredOutput(false);
+ stopTaskRunnerReq = builder.build();
+ }
+
+ @Override
+ public void stop() {
+ if(stopEventHandling.getAndSet(true)){
+ return;
+ }
+
+ if (schedulingThread != null) {
+ synchronized (schedulingThread) {
+ schedulingThread.notifyAll();
+ }
+ }
+
+ // Return all of request callbacks instantly.
+ if(taskRequests != null){
+ for (TaskRequestEvent req : taskRequests.taskRequestQueue) {
+ req.getCallback().run(stopTaskRunnerReq);
+ }
+ }
+
+ LOG.info("Task Scheduler stopped");
+ super.stop();
+ }
+
+ private Fragment[] fragmentsForNonLeafTask;
+ private Fragment[] broadcastFragmentsForNonLeafTask;
+
+ LinkedList<TaskRequestEvent> taskRequestEvents = new LinkedList<TaskRequestEvent>();
+ public void schedule() {
+
+ if (taskRequests.size() > 0) {
+ if (scheduledRequests.leafTaskNum() > 0) {
+ LOG.debug("Try to schedule tasks with taskRequestEvents: " +
+ taskRequests.size() + ", LeafTask Schedule Request: " +
+ scheduledRequests.leafTaskNum());
+ taskRequests.getTaskRequests(taskRequestEvents,
+ scheduledRequests.leafTaskNum());
+ LOG.debug("Get " + taskRequestEvents.size() + " taskRequestEvents ");
+ if (taskRequestEvents.size() > 0) {
+ scheduledRequests.assignToLeafTasks(taskRequestEvents);
+ taskRequestEvents.clear();
+ }
+ }
+ }
+
+ if (taskRequests.size() > 0) {
+ if (scheduledRequests.nonLeafTaskNum() > 0) {
+ LOG.debug("Try to schedule tasks with taskRequestEvents: " +
+ taskRequests.size() + ", NonLeafTask Schedule Request: " +
+ scheduledRequests.nonLeafTaskNum());
+ taskRequests.getTaskRequests(taskRequestEvents,
+ scheduledRequests.nonLeafTaskNum());
+ scheduledRequests.assignToNonLeafTasks(taskRequestEvents);
+ taskRequestEvents.clear();
+ }
+ }
+ }
+
+ @Override
+ public void handle(TaskSchedulerEvent event) {
+ if (event.getType() == EventType.T_SCHEDULE) {
+ if (event instanceof FragmentScheduleEvent) {
+ FragmentScheduleEvent castEvent = (FragmentScheduleEvent) event;
+ if (context.isLeafQuery()) {
+ TaskAttemptScheduleContext taskContext = new TaskAttemptScheduleContext();
+ Task task = Stage.newEmptyTask(context, taskContext, stage, nextTaskId++);
+ task.addFragment(castEvent.getLeftFragment(), true);
+ scheduledObjectNum++;
+ if (castEvent.hasRightFragments()) {
+ task.addFragments(castEvent.getRightFragments());
+ }
+ stage.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
+ } else {
+ fragmentsForNonLeafTask = new FileFragment[2];
+ fragmentsForNonLeafTask[0] = castEvent.getLeftFragment();
+ if (castEvent.hasRightFragments()) {
+ FileFragment[] rightFragments = castEvent.getRightFragments().toArray(new FileFragment[]{});
+ fragmentsForNonLeafTask[1] = rightFragments[0];
+ if (rightFragments.length > 1) {
+ broadcastFragmentsForNonLeafTask = new FileFragment[rightFragments.length - 1];
+ System.arraycopy(rightFragments, 1, broadcastFragmentsForNonLeafTask, 0, broadcastFragmentsForNonLeafTask.length);
+ } else {
+ broadcastFragmentsForNonLeafTask = null;
+ }
+ }
+ }
+ } else if (event instanceof FetchScheduleEvent) {
+ FetchScheduleEvent castEvent = (FetchScheduleEvent) event;
+ Map<String, List<FetchImpl>> fetches = castEvent.getFetches();
+ TaskAttemptScheduleContext taskScheduleContext = new TaskAttemptScheduleContext();
+ Task task = Stage.newEmptyTask(context, taskScheduleContext, stage, nextTaskId++);
+ scheduledObjectNum++;
+ for (Entry<String, List<FetchImpl>> eachFetch : fetches.entrySet()) {
+ task.addFetches(eachFetch.getKey(), eachFetch.getValue());
+ task.addFragment(fragmentsForNonLeafTask[0], true);
+ if (fragmentsForNonLeafTask[1] != null) {
+ task.addFragment(fragmentsForNonLeafTask[1], true);
+ }
+ }
+ if (broadcastFragmentsForNonLeafTask != null && broadcastFragmentsForNonLeafTask.length > 0) {
+ task.addFragments(Arrays.asList(broadcastFragmentsForNonLeafTask));
+ }
+ stage.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
+ } else if (event instanceof TaskAttemptToSchedulerEvent) {
+ TaskAttemptToSchedulerEvent castEvent = (TaskAttemptToSchedulerEvent) event;
+ if (context.isLeafQuery()) {
+ scheduledRequests.addLeafTask(castEvent);
+ } else {
+ scheduledRequests.addNonLeafTask(castEvent);
+ }
+ }
+ } else if (event.getType() == EventType.T_SCHEDULE_CANCEL) {
+ // when a stage is killed, unassigned query unit attmpts are canceled from the scheduler.
+ // This event is triggered by TaskAttempt.
+ TaskAttemptToSchedulerEvent castedEvent = (TaskAttemptToSchedulerEvent) event;
+ scheduledRequests.leafTasks.remove(castedEvent.getTaskAttempt().getId());
+ LOG.info(castedEvent.getTaskAttempt().getId() + " is canceled from " + this.getClass().getSimpleName());
+ ((TaskAttemptToSchedulerEvent) event).getTaskAttempt().handle(
+ new TaskAttemptEvent(castedEvent.getTaskAttempt().getId(), TaskAttemptEventType.TA_SCHEDULE_CANCELED));
+ }
+ }
+
+ @Override
+ public void handleTaskRequestEvent(TaskRequestEvent event) {
+
+ taskRequests.handle(event);
+ int hosts = scheduledRequests.leafTaskHostMapping.size();
+
+ // if available cluster resource are large then tasks, the scheduler thread are working immediately.
+ if(remainingScheduledObjectNum() > 0 &&
+ (remainingScheduledObjectNum() <= hosts || hosts <= taskRequests.size())){
+ synchronized (schedulingThread){
+ schedulingThread.notifyAll();
+ }
+ }
+ }
+
+ @Override
+ public int remainingScheduledObjectNum() {
+ return scheduledObjectNum;
+ }
+
+ private class TaskRequests implements EventHandler<TaskRequestEvent> {
+ private final LinkedBlockingQueue<TaskRequestEvent> taskRequestQueue =
+ new LinkedBlockingQueue<TaskRequestEvent>();
+
+ @Override
+ public void handle(TaskRequestEvent event) {
+ if(LOG.isDebugEnabled()){
+ LOG.debug("TaskRequest: " + event.getContainerId() + "," + event.getExecutionBlockId());
+ }
+
+ if(stopEventHandling.get()) {
+ event.getCallback().run(stopTaskRunnerReq);
+ return;
+ }
+ int qSize = taskRequestQueue.size();
+ if (qSize != 0 && qSize % 1000 == 0) {
+ LOG.info("Size of event-queue in DefaultTaskScheduler is " + qSize);
+ }
+ int remCapacity = taskRequestQueue.remainingCapacity();
+ if (remCapacity < 1000) {
+ LOG.warn("Very low remaining capacity in the event-queue "
+ + "of DefaultTaskScheduler: " + remCapacity);
+ }
+
+ taskRequestQueue.add(event);
+ }
+
+ public void getTaskRequests(final Collection<TaskRequestEvent> taskRequests,
+ int num) {
+ taskRequestQueue.drainTo(taskRequests, num);
+ }
+
+ public int size() {
+ return taskRequestQueue.size();
+ }
+ }
+
+ /**
+ * One worker can have multiple running task runners. <code>HostVolumeMapping</code>
+ * describes various information for one worker, including :
+ * <ul>
+ * <li>host name</li>
+ * <li>rack name</li>
+ * <li>unassigned tasks for each disk volume</li>
+ * <li>last assigned volume id - it can be used for assigning task in a round-robin manner</li>
+ * <li>the number of running tasks for each volume</li>
+ * </ul>, each task runner and the concurrency number of running tasks for volumes.
+ *
+ * Here, we identifier a task runner by {@link ContainerId}, and we use volume ids to identify
+ * all disks in this node. Actually, each volume is only used to distinguish disks, and we don't
+ * know a certain volume id indicates a certain disk. If you want to know volume id, please read the below section.
+ *
+ * <h3>Volume id</h3>
+ * Volume id is an integer. Each volume id identifies each disk volume.
+ *
+ * This volume id can be obtained from org.apache.hadoop.fs.BlockStorageLocation#getVolumeIds()}. *
+ * HDFS cannot give any volume id due to unknown reason and disabled config 'dfs.client.file-block-locations.enabled'.
+ * In this case, the volume id will be -1 or other native integer.
+ *
+ * <h3>See Also</h3>
+ * <ul>
+ * <li>HDFS-3672 (https://issues.apache.org/jira/browse/HDFS-3672).</li>
+ * </ul>
+ */
+ public class HostVolumeMapping {
+ private final String host;
+ private final String rack;
+ /** A key is disk volume, and a value is a list of tasks to be scheduled. */
+ private Map<Integer, LinkedHashSet<TaskAttempt>> unassignedTaskForEachVolume =
+ Collections.synchronizedMap(new HashMap<Integer, LinkedHashSet<TaskAttempt>>());
+ /** A value is last assigned volume id for each task runner */
+ private HashMap<TajoContainerId, Integer> lastAssignedVolumeId = new HashMap<TajoContainerId,
+ Integer>();
+ /**
+ * A key is disk volume id, and a value is the load of this volume.
+ * This load is measured by counting how many number of tasks are running.
+ *
+ * These disk volumes are kept in an order of ascending order of the volume id.
+ * In other words, the head volume ids are likely to -1, meaning no given volume id.
+ */
+ private SortedMap<Integer, Integer> diskVolumeLoads = new TreeMap<Integer, Integer>();
+ /** The total number of remain tasks in this host */
+ private AtomicInteger remainTasksNum = new AtomicInteger(0);
+ public static final int REMOTE = -2;
+
+
+ public HostVolumeMapping(String host, String rack){
+ this.host = host;
+ this.rack = rack;
+ }
+
+ public synchronized void addTaskAttempt(int volumeId, TaskAttempt attemptId){
+ synchronized (unassignedTaskForEachVolume){
+ LinkedHashSet<TaskAttempt> list = unassignedTaskForEachVolume.get(volumeId);
+ if (list == null) {
+ list = new LinkedHashSet<TaskAttempt>();
+ unassignedTaskForEachVolume.put(volumeId, list);
+ }
+ list.add(attemptId);
+ }
+
+ remainTasksNum.incrementAndGet();
+
+ if(!diskVolumeLoads.containsKey(volumeId)) diskVolumeLoads.put(volumeId, 0);
+ }
+
+ /**
+ * Priorities
+ * 1. a task list in a volume of host
+ * 2. unknown block or Non-splittable task in host
+ * 3. remote tasks. unassignedTaskForEachVolume is only contained local task. so it will be null
+ */
+ public synchronized TaskAttemptId getLocalTask(TajoContainerId containerId) {
+ int volumeId;
+ TaskAttemptId taskAttemptId = null;
+
+ if (!lastAssignedVolumeId.containsKey(containerId)) {
+ volumeId = getLowestVolumeId();
+ increaseConcurrency(containerId, volumeId);
+ } else {
+ volumeId = lastAssignedVolumeId.get(containerId);
+ }
+
+ if (unassignedTaskForEachVolume.size() > 0) {
+ int retry = unassignedTaskForEachVolume.size();
+ do {
+ //clean and get a remaining local task
+ taskAttemptId = getAndRemove(volumeId);
+ if(!unassignedTaskForEachVolume.containsKey(volumeId)) {
+ decreaseConcurrency(containerId);
+ if (volumeId > REMOTE) {
+ diskVolumeLoads.remove(volumeId);
+ }
+ }
+
+ if (taskAttemptId == null) {
+ //reassign next volume
+ volumeId = getLowestVolumeId();
+ increaseConcurrency(containerId, volumeId);
+ retry--;
+ } else {
+ break;
+ }
+ } while (retry > 0);
+ } else {
+ this.remainTasksNum.set(0);
+ }
+ return taskAttemptId;
+ }
+
+ public synchronized TaskAttemptId getTaskAttemptIdByRack(String rack) {
+ TaskAttemptId taskAttemptId = null;
+
+ if (unassignedTaskForEachVolume.size() > 0 && this.rack.equals(rack)) {
+ int retry = unassignedTaskForEachVolume.size();
+ do {
+ //clean and get a remaining task
+ int volumeId = getLowestVolumeId();
+ taskAttemptId = getAndRemove(volumeId);
+ if (taskAttemptId == null) {
+ if (volumeId > REMOTE) {
+ diskVolumeLoads.remove(volumeId);
+ }
+ retry--;
+ } else {
+ break;
+ }
+ } while (retry > 0);
+ }
+ return taskAttemptId;
+ }
+
+ private synchronized TaskAttemptId getAndRemove(int volumeId){
+ TaskAttemptId taskAttemptId = null;
+ if(!unassignedTaskForEachVolume.containsKey(volumeId)) return taskAttemptId;
+
+ LinkedHashSet<TaskAttempt> list = unassignedTaskForEachVolume.get(volumeId);
+ if(list != null && list.size() > 0){
+ TaskAttempt taskAttempt;
+ synchronized (unassignedTaskForEachVolume) {
+ Iterator<TaskAttempt> iterator = list.iterator();
+ taskAttempt = iterator.next();
+ iterator.remove();
+ }
+
+ this.remainTasksNum.getAndDecrement();
+ taskAttemptId = taskAttempt.getId();
+ for (DataLocation location : taskAttempt.getTask().getDataLocations()) {
+ if (!this.getHost().equals(location.getHost())) {
+ HostVolumeMapping volumeMapping = scheduledRequests.leafTaskHostMapping.get(location.getHost());
+ if (volumeMapping != null) {
+ volumeMapping.removeTaskAttempt(location.getVolumeId(), taskAttempt);
+ }
+ }
+ }
+ }
+
+ if(list == null || list.isEmpty()) {
+ unassignedTaskForEachVolume.remove(volumeId);
+ }
+ return taskAttemptId;
+ }
+
+ private synchronized void removeTaskAttempt(int volumeId, TaskAttempt taskAttempt){
+ if(!unassignedTaskForEachVolume.containsKey(volumeId)) return;
+
+ LinkedHashSet<TaskAttempt> tasks = unassignedTaskForEachVolume.get(volumeId);
+
+ if(tasks != null && tasks.size() > 0){
+ tasks.remove(taskAttempt);
+ remainTasksNum.getAndDecrement();
+ } else {
+ unassignedTaskForEachVolume.remove(volumeId);
+ }
+ }
+
+ /**
+ * Increase the count of running tasks and disk loads for a certain task runner.
+ *
+ * @param containerId The task runner identifier
+ * @param volumeId Volume identifier
+ * @return the volume load (i.e., how many running tasks use this volume)
+ */
+ private synchronized int increaseConcurrency(TajoContainerId containerId, int volumeId) {
+
+ int concurrency = 1;
+ if (diskVolumeLoads.containsKey(volumeId)) {
+ concurrency = diskVolumeLoads.get(volumeId) + 1;
+ }
+
+ if (volumeId > -1) {
+ LOG.info("Assigned host : " + host + ", Volume : " + volumeId + ", Concurrency : " + concurrency);
+ } else if (volumeId == -1) {
+ // this case is disabled namenode block meta or compressed text file or amazon s3
+ LOG.info("Assigned host : " + host + ", Unknown Volume : " + volumeId + ", Concurrency : " + concurrency);
+ } else if (volumeId == REMOTE) {
+ // this case has processed all block on host and it will be assigned to remote
+ LOG.info("Assigned host : " + host + ", Remaining local tasks : " + getRemainingLocalTaskSize()
+ + ", Remote Concurrency : " + concurrency);
+ }
+ diskVolumeLoads.put(volumeId, concurrency);
+ lastAssignedVolumeId.put(containerId, volumeId);
+ return concurrency;
+ }
+
+ /**
+ * Decrease the count of running tasks of a certain task runner
+ */
+ private synchronized void decreaseConcurrency(TajoContainerId containerId){
+ Integer volumeId = lastAssignedVolumeId.get(containerId);
+ if(volumeId != null && diskVolumeLoads.containsKey(volumeId)){
+ Integer concurrency = diskVolumeLoads.get(volumeId);
+ if(concurrency > 0){
+ diskVolumeLoads.put(volumeId, concurrency - 1);
+ } else {
+ if (volumeId > REMOTE) {
+ diskVolumeLoads.remove(volumeId);
+ }
+ }
+ }
+ lastAssignedVolumeId.remove(containerId);
+ }
+
+ /**
+ * volume of a host : 0 ~ n
+ * compressed task, amazon s3, unKnown volume : -1
+ * remote task : -2
+ */
+ public int getLowestVolumeId(){
+ Map.Entry<Integer, Integer> volumeEntry = null;
+
+ for (Map.Entry<Integer, Integer> entry : diskVolumeLoads.entrySet()) {
+ if(volumeEntry == null) volumeEntry = entry;
+
+ if (volumeEntry.getValue() >= entry.getValue()) {
+ volumeEntry = entry;
+ }
+ }
+
+ if(volumeEntry != null){
+ return volumeEntry.getKey();
+ } else {
+ return REMOTE;
+ }
+ }
+
+ public boolean isAssigned(TajoContainerId containerId){
+ return lastAssignedVolumeId.containsKey(containerId);
+ }
+
+ public boolean isRemote(TajoContainerId containerId){
+ Integer volumeId = lastAssignedVolumeId.get(containerId);
+ if(volumeId == null || volumeId > REMOTE){
+ return false;
+ } else {
+ return true;
+ }
+ }
+
+ public int getRemoteConcurrency(){
+ return getVolumeConcurrency(REMOTE);
+ }
+
+ public int getVolumeConcurrency(int volumeId){
+ Integer size = diskVolumeLoads.get(volumeId);
+ if(size == null) return 0;
+ else return size;
+ }
+
+ public int getRemainingLocalTaskSize(){
+ return remainTasksNum.get();
+ }
+
+ public String getHost() {
+
+ return host;
+ }
+
+ public String getRack() {
+ return rack;
+ }
+ }
+
+ private class ScheduledRequests {
+ // two list leafTasks and nonLeafTasks keep all tasks to be scheduled. Even though some task is included in
+ // leafTaskHostMapping or leafTasksRackMapping, some task T will not be sent to a task runner
+ // if the task is not included in leafTasks and nonLeafTasks.
+ private final Set<TaskAttemptId> leafTasks = Collections.synchronizedSet(new HashSet<TaskAttemptId>());
+ private final Set<TaskAttemptId> nonLeafTasks = Collections.synchronizedSet(new HashSet<TaskAttemptId>());
+ private Map<String, HostVolumeMapping> leafTaskHostMapping = Maps.newConcurrentMap();
+ private final Map<String, HashSet<TaskAttemptId>> leafTasksRackMapping = Maps.newConcurrentMap();
+
+ private synchronized void addLeafTask(TaskAttemptToSchedulerEvent event) {
+ TaskAttempt taskAttempt = event.getTaskAttempt();
+ List<DataLocation> locations = taskAttempt.getTask().getDataLocations();
+
+ for (DataLocation location : locations) {
+ String host = location.getHost();
+
+ HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host);
+ if (hostVolumeMapping == null) {
+ String rack = RackResolver.resolve(host).getNetworkLocation();
+ hostVolumeMapping = new HostVolumeMapping(host, rack);
+ leafTaskHostMapping.put(host, hostVolumeMapping);
+ }
+ hostVolumeMapping.addTaskAttempt(location.getVolumeId(), taskAttempt);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Added attempt req to host " + host);
+ }
+
+ HashSet<TaskAttemptId> list = leafTasksRackMapping.get(hostVolumeMapping.getRack());
+ if (list == null) {
+ list = new HashSet<TaskAttemptId>();
+ leafTasksRackMapping.put(hostVolumeMapping.getRack(), list);
+ }
+
+ list.add(taskAttempt.getId());
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Added attempt req to rack " + hostVolumeMapping.getRack());
+ }
+ }
+
+ leafTasks.add(taskAttempt.getId());
+ }
+
+ private void addNonLeafTask(TaskAttemptToSchedulerEvent event) {
+ nonLeafTasks.add(event.getTaskAttempt().getId());
+ }
+
+ public int leafTaskNum() {
+ return leafTasks.size();
+ }
+
+ public int nonLeafTaskNum() {
+ return nonLeafTasks.size();
+ }
+
+ public Set<TaskAttemptId> assignedRequest = new HashSet<TaskAttemptId>();
+
+ private TaskAttemptId allocateLocalTask(String host, TajoContainerId containerId){
+ HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host);
+
+ if (hostVolumeMapping != null) { //tajo host is located in hadoop datanode
+ for (int i = 0; i < hostVolumeMapping.getRemainingLocalTaskSize(); i++) {
+ TaskAttemptId attemptId = hostVolumeMapping.getLocalTask(containerId);
+
+ if(attemptId == null) break;
+ //find remaining local task
+ if (leafTasks.contains(attemptId)) {
+ leafTasks.remove(attemptId);
+ //LOG.info(attemptId + " Assigned based on host match " + hostName);
+ hostLocalAssigned++;
+ totalAssigned++;
+ return attemptId;
+ }
+ }
+ }
+ return null;
+ }
+
+ private TaskAttemptId allocateRackTask(String host) {
+
+ List<HostVolumeMapping> remainingTasks = Lists.newArrayList(leafTaskHostMapping.values());
+ String rack = RackResolver.resolve(host).getNetworkLocation();
+ TaskAttemptId attemptId = null;
+
+ if (remainingTasks.size() > 0) {
+ synchronized (scheduledRequests) {
+ //find largest remaining task of other host in rack
+ Collections.sort(remainingTasks, new Comparator<HostVolumeMapping>() {
+ @Override
+ public int compare(HostVolumeMapping v1, HostVolumeMapping v2) {
+ // descending remaining tasks
+ if (v2.remainTasksNum.get() > v1.remainTasksNum.get()) {
+ return 1;
+ } else if (v2.remainTasksNum.get() == v1.remainTasksNum.get()) {
+ return 0;
+ } else {
+ return -1;
+ }
+ }
+ });
+ }
+
+ for (HostVolumeMapping tasks : remainingTasks) {
+ for (int i = 0; i < tasks.getRemainingLocalTaskSize(); i++) {
+ TaskAttemptId tId = tasks.getTaskAttemptIdByRack(rack);
+
+ if (tId == null) break;
+
+ if (leafTasks.contains(tId)) {
+ leafTasks.remove(tId);
+ attemptId = tId;
+ break;
+ }
+ }
+ if(attemptId != null) break;
+ }
+ }
+
+ //find task in rack
+ if (attemptId == null) {
+ HashSet<TaskAttemptId> list = leafTasksRackMapping.get(rack);
+ if (list != null) {
+ synchronized (list) {
+ Iterator<TaskAttemptId> iterator = list.iterator();
+ while (iterator.hasNext()) {
+ TaskAttemptId tId = iterator.next();
+ iterator.remove();
+ if (leafTasks.contains(tId)) {
+ leafTasks.remove(tId);
+ attemptId = tId;
+ break;
+ }
+ }
+ }
+ }
+ }
+
+ if (attemptId != null) {
+ rackLocalAssigned++;
+ totalAssigned++;
+
+ LOG.info(String.format("Assigned Local/Rack/Total: (%d/%d/%d), Locality: %.2f%%, Rack host: %s",
+ hostLocalAssigned, rackLocalAssigned, totalAssigned,
+ ((double) hostLocalAssigned / (double) totalAssigned) * 100, host));
+
+ }
+ return attemptId;
+ }
+
+ public void assignToLeafTasks(LinkedList<TaskRequestEvent> taskRequests) {
+ Collections.shuffle(taskRequests);
+ LinkedList<TaskRequestEvent> remoteTaskRequests = new LinkedList<TaskRequestEvent>();
+
+ TaskRequestEvent taskRequest;
+ while (leafTasks.size() > 0 && (!taskRequests.isEmpty() || !remoteTaskRequests.isEmpty())) {
+ taskRequest = taskRequests.pollFirst();
+ if(taskRequest == null) { // if there are only remote task requests
+ taskRequest = remoteTaskRequests.pollFirst();
+ }
+
+ // checking if this container is still alive.
+ // If not, ignore the task request and stop the task runner
+ ContainerProxy container = context.getMasterContext().getResourceAllocator()
+ .getContainer(taskRequest.getContainerId());
+ if(container == null) {
+ taskRequest.getCallback().run(stopTaskRunnerReq);
+ continue;
+ }
+
+ // getting the hostname of requested node
+ WorkerConnectionInfo connectionInfo =
+ context.getMasterContext().getResourceAllocator().getWorkerConnectionInfo(taskRequest.getWorkerId());
+ String host = connectionInfo.getHost();
+
+ // if there are no worker matched to the hostname a task request
+ if(!leafTaskHostMapping.containsKey(host)){
+ String normalizedHost = NetUtils.normalizeHost(host);
+
+ if(!leafTaskHostMapping.containsKey(normalizedHost) && !taskRequests.isEmpty()){
+ // this case means one of either cases:
+ // * there are no blocks which reside in this node.
+ // * all blocks which reside in this node are consumed, and this task runner requests a remote task.
+ // In this case, we transfer the task request to the remote task request list, and skip the followings.
+ remoteTaskRequests.add(taskRequest);
+ continue;
+ }
+ }
+
+ TajoContainerId containerId = taskRequest.getContainerId();
+ LOG.debug("assignToLeafTasks: " + taskRequest.getExecutionBlockId() + "," +
+ "containerId=" + containerId);
+
+ //////////////////////////////////////////////////////////////////////
+ // disk or host-local allocation
+ //////////////////////////////////////////////////////////////////////
+ TaskAttemptId attemptId = allocateLocalTask(host, containerId);
+
+ if (attemptId == null) { // if a local task cannot be found
+ HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host);
+
+ if(hostVolumeMapping != null) {
+ if(!hostVolumeMapping.isRemote(containerId)){
+ // assign to remote volume
+ hostVolumeMapping.decreaseConcurrency(containerId);
+ hostVolumeMapping.increaseConcurrency(containerId, HostVolumeMapping.REMOTE);
+ }
+ // this part is remote concurrency management of a tail tasks
+ int tailLimit = Math.max(remainingScheduledObjectNum() / (leafTaskHostMapping.size() * 2), 1);
+
+ if(hostVolumeMapping.getRemoteConcurrency() > tailLimit){
+ //release container
+ hostVolumeMapping.decreaseConcurrency(containerId);
+ taskRequest.getCallback().run(stopTaskRunnerReq);
+ continue;
+ }
+ }
+
+ //////////////////////////////////////////////////////////////////////
+ // rack-local allocation
+ //////////////////////////////////////////////////////////////////////
+ attemptId = allocateRackTask(host);
+
+ //////////////////////////////////////////////////////////////////////
+ // random node allocation
+ //////////////////////////////////////////////////////////////////////
+ if (attemptId == null && leafTaskNum() > 0) {
+ synchronized (leafTasks){
+ attemptId = leafTasks.iterator().next();
+ leafTasks.remove(attemptId);
+ rackLocalAssigned++;
+ totalAssigned++;
+ LOG.info(String.format("Assigned Local/Remote/Total: (%d/%d/%d), Locality: %.2f%%,",
+ hostLocalAssigned, rackLocalAssigned, totalAssigned,
+ ((double) hostLocalAssigned / (double) totalAssigned) * 100));
+ }
+ }
+ }
+
+ if (attemptId != null) {
+ Task task = stage.getTask(attemptId.getTaskId());
+ TaskRequest taskAssign = new TaskRequestImpl(
+ attemptId,
+ new ArrayList<FragmentProto>(task.getAllFragments()),
+ "",
+ false,
+ LogicalNodeSerializer.serialize(task.getLogicalPlan()),
+ context.getMasterContext().getQueryContext(),
+ stage.getDataChannel(), stage.getBlock().getEnforcer());
+ if (checkIfInterQuery(stage.getMasterPlan(), stage.getBlock())) {
+ taskAssign.setInterQuery();
+ }
+
+ context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId,
+ taskRequest.getContainerId(), connectionInfo));
+ assignedRequest.add(attemptId);
+
+ scheduledObjectNum--;
+ taskRequest.getCallback().run(taskAssign.getProto());
+ } else {
+ throw new RuntimeException("Illegal State!!!!!!!!!!!!!!!!!!!!!");
+ }
+ }
+ }
+
+ private boolean checkIfInterQuery(MasterPlan masterPlan, ExecutionBlock block) {
+ if (masterPlan.isRoot(block)) {
+ return false;
+ }
+
+ ExecutionBlock parent = masterPlan.getParent(block);
+ if (masterPlan.isRoot(parent) && parent.hasUnion()) {
+ return false;
+ }
+
+ return true;
+ }
+
+ public void assignToNonLeafTasks(LinkedList<TaskRequestEvent> taskRequests) {
+ Collections.shuffle(taskRequests);
+
+ TaskRequestEvent taskRequest;
+ while (!taskRequests.isEmpty()) {
+ taskRequest = taskRequests.pollFirst();
+ LOG.debug("assignToNonLeafTasks: " + taskRequest.getExecutionBlockId());
+
+ TaskAttemptId attemptId;
+ // random allocation
+ if (nonLeafTasks.size() > 0) {
+ synchronized (nonLeafTasks){
+ attemptId = nonLeafTasks.iterator().next();
+ nonLeafTasks.remove(attemptId);
+ }
+ LOG.debug("Assigned based on * match");
+
+ Task task;
+ task = stage.getTask(attemptId.getTaskId());
+ TaskRequest taskAssign = new TaskRequestImpl(
+ attemptId,
+ Lists.newArrayList(task.getAllFragments()),
+ "",
+ false,
+ LogicalNodeSerializer.serialize(task.getLogicalPlan()),
+ context.getMasterContext().getQueryContext(),
+ stage.getDataChannel(),
+ stage.getBlock().getEnforcer());
+ if (checkIfInterQuery(stage.getMasterPlan(), stage.getBlock())) {
+ taskAssign.setInterQuery();
+ }
+ for(Map.Entry<String, Set<FetchImpl>> entry: task.getFetchMap().entrySet()) {
+ Collection<FetchImpl> fetches = entry.getValue();
+ if (fetches != null) {
+ for (FetchImpl fetch : fetches) {
+ taskAssign.addFetch(entry.getKey(), fetch);
+ }
+ }
+ }
+
+ WorkerConnectionInfo connectionInfo = context.getMasterContext().getResourceAllocator().
+ getWorkerConnectionInfo(taskRequest.getWorkerId());
+ context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId,
+ taskRequest.getContainerId(), connectionInfo));
+ taskRequest.getCallback().run(taskAssign.getProto());
+ totalAssigned++;
+ scheduledObjectNum--;
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/FetchScheduleEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/FetchScheduleEvent.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/FetchScheduleEvent.java
new file mode 100644
index 0000000..5fe2f80
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/FetchScheduleEvent.java
@@ -0,0 +1,40 @@
+/*
+ * 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.tajo.querymaster;
+
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.master.event.TaskSchedulerEvent;
+import org.apache.tajo.worker.FetchImpl;
+
+import java.util.List;
+import java.util.Map;
+
+public class FetchScheduleEvent extends TaskSchedulerEvent {
+ private final Map<String, List<FetchImpl>> fetches;
+
+ public FetchScheduleEvent(final EventType eventType, final ExecutionBlockId blockId,
+ final Map<String, List<FetchImpl>> fetches) {
+ super(eventType, blockId);
+ this.fetches = fetches;
+ }
+
+ public Map<String, List<FetchImpl>> getFetches() {
+ return fetches;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
new file mode 100644
index 0000000..2932694
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java
@@ -0,0 +1,738 @@
+/**
+ * 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.tajo.querymaster;
+
+import com.google.common.collect.Maps;
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.state.*;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.TajoProtos.QueryState;
+import org.apache.tajo.catalog.proto.CatalogProtos.UpdateTableStatsProto;
+import org.apache.tajo.catalog.CatalogService;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.global.ExecutionBlock;
+import org.apache.tajo.engine.planner.global.ExecutionBlockCursor;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.plan.logical.*;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.plan.util.PlannerUtil;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.StorageConstants;
+import org.apache.tajo.util.TUtil;
+import org.apache.tajo.util.history.QueryHistory;
+import org.apache.tajo.util.history.StageHistory;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+public class Query implements EventHandler<QueryEvent> {
+ private static final Log LOG = LogFactory.getLog(Query.class);
+
+ // Facilities for Query
+ private final TajoConf systemConf;
+ private final Clock clock;
+ private String queryStr;
+ private Map<ExecutionBlockId, Stage> stages;
+ private final EventHandler eventHandler;
+ private final MasterPlan plan;
+ QueryMasterTask.QueryMasterTaskContext context;
+ private ExecutionBlockCursor cursor;
+
+ // Query Status
+ private final QueryId id;
+ private long appSubmitTime;
+ private long startTime;
+ private long finishTime;
+ private TableDesc resultDesc;
+ private int completedStagesCount = 0;
+ private int successedStagesCount = 0;
+ private int killedStagesCount = 0;
+ private int failedStagesCount = 0;
+ private int erroredStagesCount = 0;
+ private final List<String> diagnostics = new ArrayList<String>();
+
+ // Internal Variables
+ private final Lock readLock;
+ private final Lock writeLock;
+ private int priority = 100;
+
+ // State Machine
+ private final StateMachine<QueryState, QueryEventType, QueryEvent> stateMachine;
+ private QueryState queryState;
+
+ // Transition Handler
+ private static final SingleArcTransition INTERNAL_ERROR_TRANSITION = new InternalErrorTransition();
+ private static final DiagnosticsUpdateTransition DIAGNOSTIC_UPDATE_TRANSITION = new DiagnosticsUpdateTransition();
+ private static final StageCompletedTransition STAGE_COMPLETED_TRANSITION = new StageCompletedTransition();
+ private static final QueryCompletedTransition QUERY_COMPLETED_TRANSITION = new QueryCompletedTransition();
+
+ protected static final StateMachineFactory
+ <Query,QueryState,QueryEventType,QueryEvent> stateMachineFactory =
+ new StateMachineFactory<Query, QueryState, QueryEventType, QueryEvent>
+ (QueryState.QUERY_NEW)
+
+ // Transitions from NEW state
+ .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_RUNNING,
+ QueryEventType.START,
+ new StartTransition())
+ .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_NEW,
+ QueryEventType.DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_KILLED,
+ QueryEventType.KILL,
+ new KillNewQueryTransition())
+ .addTransition(QueryState.QUERY_NEW, QueryState.QUERY_ERROR,
+ QueryEventType.INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+
+ // Transitions from RUNNING state
+ .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_RUNNING,
+ QueryEventType.STAGE_COMPLETED,
+ STAGE_COMPLETED_TRANSITION)
+ .addTransition(QueryState.QUERY_RUNNING,
+ EnumSet.of(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_FAILED, QueryState.QUERY_KILLED,
+ QueryState.QUERY_ERROR),
+ QueryEventType.QUERY_COMPLETED,
+ QUERY_COMPLETED_TRANSITION)
+ .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_RUNNING,
+ QueryEventType.DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_KILL_WAIT,
+ QueryEventType.KILL,
+ new KillAllStagesTransition())
+ .addTransition(QueryState.QUERY_RUNNING, QueryState.QUERY_ERROR,
+ QueryEventType.INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+
+ // Transitions from QUERY_SUCCEEDED state
+ .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_SUCCEEDED,
+ QueryEventType.DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ // ignore-able transitions
+ .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_SUCCEEDED,
+ QueryEventType.STAGE_COMPLETED,
+ STAGE_COMPLETED_TRANSITION)
+ .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_SUCCEEDED,
+ QueryEventType.KILL)
+ .addTransition(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_ERROR,
+ QueryEventType.INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+
+ // Transitions from KILL_WAIT state
+ .addTransition(QueryState.QUERY_KILL_WAIT, QueryState.QUERY_KILL_WAIT,
+ QueryEventType.STAGE_COMPLETED,
+ STAGE_COMPLETED_TRANSITION)
+ .addTransition(QueryState.QUERY_KILL_WAIT,
+ EnumSet.of(QueryState.QUERY_SUCCEEDED, QueryState.QUERY_FAILED, QueryState.QUERY_KILLED,
+ QueryState.QUERY_ERROR),
+ QueryEventType.QUERY_COMPLETED,
+ QUERY_COMPLETED_TRANSITION)
+ .addTransition(QueryState.QUERY_KILL_WAIT, QueryState.QUERY_KILL_WAIT,
+ QueryEventType.DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(QueryState.QUERY_KILL_WAIT, QueryState.QUERY_ERROR,
+ QueryEventType.INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+ // Ignore-able transitions
+ .addTransition(QueryState.QUERY_KILL_WAIT, EnumSet.of(QueryState.QUERY_KILLED),
+ QueryEventType.KILL,
+ QUERY_COMPLETED_TRANSITION)
+
+ // Transitions from FAILED state
+ .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_FAILED,
+ QueryEventType.DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_ERROR,
+ QueryEventType.INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+ // Ignore-able transitions
+ .addTransition(QueryState.QUERY_FAILED, QueryState.QUERY_FAILED,
+ QueryEventType.KILL)
+
+ // Transitions from ERROR state
+ .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
+ QueryEventType.DIAGNOSTIC_UPDATE,
+ DIAGNOSTIC_UPDATE_TRANSITION)
+ .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
+ QueryEventType.INTERNAL_ERROR,
+ INTERNAL_ERROR_TRANSITION)
+ // Ignore-able transitions
+ .addTransition(QueryState.QUERY_ERROR, QueryState.QUERY_ERROR,
+ EnumSet.of(QueryEventType.KILL, QueryEventType.STAGE_COMPLETED))
+
+ .installTopology();
+
+ public Query(final QueryMasterTask.QueryMasterTaskContext context, final QueryId id,
+ final long appSubmitTime,
+ final String queryStr,
+ final EventHandler eventHandler,
+ final MasterPlan plan) {
+ this.context = context;
+ this.systemConf = context.getConf();
+ this.id = id;
+ this.clock = context.getClock();
+ this.appSubmitTime = appSubmitTime;
+ this.queryStr = queryStr;
+ this.stages = Maps.newConcurrentMap();
+ this.eventHandler = eventHandler;
+ this.plan = plan;
+ this.cursor = new ExecutionBlockCursor(plan, true);
+
+ StringBuilder sb = new StringBuilder("\n=======================================================");
+ sb.append("\nThe order of execution: \n");
+ int order = 1;
+ while (cursor.hasNext()) {
+ ExecutionBlock currentEB = cursor.nextBlock();
+ sb.append("\n").append(order).append(": ").append(currentEB.getId());
+ order++;
+ }
+ sb.append("\n=======================================================");
+ LOG.info(sb);
+ cursor.reset();
+
+ ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+ this.readLock = readWriteLock.readLock();
+ this.writeLock = readWriteLock.writeLock();
+
+ stateMachine = stateMachineFactory.make(this);
+ queryState = stateMachine.getCurrentState();
+ }
+
+ public float getProgress() {
+ QueryState state = getState();
+ if (state == QueryState.QUERY_SUCCEEDED) {
+ return 1.0f;
+ } else {
+ int idx = 0;
+ List<Stage> tempStages = new ArrayList<Stage>();
+ synchronized(stages) {
+ tempStages.addAll(stages.values());
+ }
+
+ float [] subProgresses = new float[tempStages.size()];
+ for (Stage stage: tempStages) {
+ if (stage.getState() != StageState.NEW) {
+ subProgresses[idx] = stage.getProgress();
+ } else {
+ subProgresses[idx] = 0.0f;
+ }
+ idx++;
+ }
+
+ float totalProgress = 0.0f;
+ float proportion = 1.0f / (float)(getExecutionBlockCursor().size() - 1); // minus one is due to
+
+ for (int i = 0; i < subProgresses.length; i++) {
+ totalProgress += subProgresses[i] * proportion;
+ }
+
+ return totalProgress;
+ }
+ }
+
+ public long getAppSubmitTime() {
+ return this.appSubmitTime;
+ }
+
+ public long getStartTime() {
+ return startTime;
+ }
+
+ public void setStartTime() {
+ startTime = clock.getTime();
+ }
+
+ public long getFinishTime() {
+ return finishTime;
+ }
+
+ public void setFinishTime() {
+ finishTime = clock.getTime();
+ }
+
+ public QueryHistory getQueryHistory() {
+ QueryHistory queryHistory = makeQueryHistory();
+ queryHistory.setStageHistories(makeStageHistories());
+ return queryHistory;
+ }
+
+ private List<StageHistory> makeStageHistories() {
+ List<StageHistory> stageHistories = new ArrayList<StageHistory>();
+ for(Stage eachStage : getStages()) {
+ stageHistories.add(eachStage.getStageHistory());
+ }
+
+ return stageHistories;
+ }
+
+ private QueryHistory makeQueryHistory() {
+ QueryHistory queryHistory = new QueryHistory();
+
+ queryHistory.setQueryId(getId().toString());
+ queryHistory.setQueryMaster(context.getQueryMasterContext().getWorkerContext().getWorkerName());
+ queryHistory.setHttpPort(context.getQueryMasterContext().getWorkerContext().getConnectionInfo().getHttpInfoPort());
+ queryHistory.setLogicalPlan(plan.toString());
+ queryHistory.setLogicalPlan(plan.getLogicalPlan().toString());
+ queryHistory.setDistributedPlan(plan.toString());
+
+ List<String[]> sessionVariables = new ArrayList<String[]>();
+ for(Map.Entry<String,String> entry: plan.getContext().getAllKeyValus().entrySet()) {
+ if (SessionVars.exists(entry.getKey()) && SessionVars.isPublic(SessionVars.get(entry.getKey()))) {
+ sessionVariables.add(new String[]{entry.getKey(), entry.getValue()});
+ }
+ }
+ queryHistory.setSessionVariables(sessionVariables);
+
+ return queryHistory;
+ }
+
+ public List<String> getDiagnostics() {
+ readLock.lock();
+ try {
+ return diagnostics;
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ protected void addDiagnostic(String diag) {
+ diagnostics.add(diag);
+ }
+
+ public TableDesc getResultDesc() {
+ return resultDesc;
+ }
+
+ public void setResultDesc(TableDesc desc) {
+ resultDesc = desc;
+ }
+
+ public MasterPlan getPlan() {
+ return plan;
+ }
+
+ public StateMachine<QueryState, QueryEventType, QueryEvent> getStateMachine() {
+ return stateMachine;
+ }
+
+ public void addStage(Stage stage) {
+ stages.put(stage.getId(), stage);
+ }
+
+ public QueryId getId() {
+ return this.id;
+ }
+
+ public Stage getStage(ExecutionBlockId id) {
+ return this.stages.get(id);
+ }
+
+ public Collection<Stage> getStages() {
+ return this.stages.values();
+ }
+
+ public QueryState getSynchronizedState() {
+ readLock.lock();
+ try {
+ return stateMachine.getCurrentState();
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ /* non-blocking call for client API */
+ public QueryState getState() {
+ return queryState;
+ }
+
+ public ExecutionBlockCursor getExecutionBlockCursor() {
+ return cursor;
+ }
+
+ public static class StartTransition
+ implements SingleArcTransition<Query, QueryEvent> {
+
+ @Override
+ public void transition(Query query, QueryEvent queryEvent) {
+
+ query.setStartTime();
+ Stage stage = new Stage(query.context, query.getPlan(),
+ query.getExecutionBlockCursor().nextBlock());
+ stage.setPriority(query.priority--);
+ query.addStage(stage);
+
+ stage.handle(new StageEvent(stage.getId(), StageEventType.SQ_INIT));
+ LOG.debug("Schedule unit plan: \n" + stage.getBlock().getPlan());
+ }
+ }
+
+ public static class QueryCompletedTransition implements MultipleArcTransition<Query, QueryEvent, QueryState> {
+
+ @Override
+ public QueryState transition(Query query, QueryEvent queryEvent) {
+ QueryCompletedEvent stageEvent = (QueryCompletedEvent) queryEvent;
+ QueryState finalState;
+
+ if (stageEvent.getState() == StageState.SUCCEEDED) {
+ finalState = finalizeQuery(query, stageEvent);
+ } else if (stageEvent.getState() == StageState.FAILED) {
+ finalState = QueryState.QUERY_FAILED;
+ } else if (stageEvent.getState() == StageState.KILLED) {
+ finalState = QueryState.QUERY_KILLED;
+ } else {
+ finalState = QueryState.QUERY_ERROR;
+ }
+ if (finalState != QueryState.QUERY_SUCCEEDED) {
+ Stage lastStage = query.getStage(stageEvent.getExecutionBlockId());
+ if (lastStage != null && lastStage.getTableMeta() != null) {
+ StoreType storeType = lastStage.getTableMeta().getStoreType();
+ if (storeType != null) {
+ LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
+ try {
+ StorageManager.getStorageManager(query.systemConf, storeType).rollbackOutputCommit(rootNode.getChild());
+ } catch (IOException e) {
+ LOG.warn(query.getId() + ", failed processing cleanup storage when query failed:" + e.getMessage(), e);
+ }
+ }
+ }
+ }
+ query.eventHandler.handle(new QueryMasterQueryCompletedEvent(query.getId()));
+ query.setFinishTime();
+
+ return finalState;
+ }
+
+ private QueryState finalizeQuery(Query query, QueryCompletedEvent event) {
+ Stage lastStage = query.getStage(event.getExecutionBlockId());
+ StoreType storeType = lastStage.getTableMeta().getStoreType();
+ try {
+ LogicalRootNode rootNode = lastStage.getMasterPlan().getLogicalPlan().getRootBlock().getRoot();
+ CatalogService catalog = lastStage.getContext().getQueryMasterContext().getWorkerContext().getCatalog();
+ TableDesc tableDesc = PlannerUtil.getTableDesc(catalog, rootNode.getChild());
+
+ Path finalOutputDir = StorageManager.getStorageManager(query.systemConf, storeType)
+ .commitOutputData(query.context.getQueryContext(),
+ lastStage.getId(), lastStage.getMasterPlan().getLogicalPlan(), lastStage.getSchema(), tableDesc);
+
+ QueryHookExecutor hookExecutor = new QueryHookExecutor(query.context.getQueryMasterContext());
+ hookExecutor.execute(query.context.getQueryContext(), query, event.getExecutionBlockId(), finalOutputDir);
+ } catch (Exception e) {
+ query.eventHandler.handle(new QueryDiagnosticsUpdateEvent(query.id, ExceptionUtils.getStackTrace(e)));
+ return QueryState.QUERY_ERROR;
+ }
+
+ return QueryState.QUERY_SUCCEEDED;
+ }
+
+ private static interface QueryHook {
+ boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir);
+ void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext, Query query,
+ ExecutionBlockId finalExecBlockId, Path finalOutputDir) throws Exception;
+ }
+
+ private class QueryHookExecutor {
+ private List<QueryHook> hookList = TUtil.newList();
+ private QueryMaster.QueryMasterContext context;
+
+ public QueryHookExecutor(QueryMaster.QueryMasterContext context) {
+ this.context = context;
+ hookList.add(new MaterializedResultHook());
+ hookList.add(new CreateTableHook());
+ hookList.add(new InsertTableHook());
+ }
+
+ public void execute(QueryContext queryContext, Query query,
+ ExecutionBlockId finalExecBlockId,
+ Path finalOutputDir) throws Exception {
+ for (QueryHook hook : hookList) {
+ if (hook.isEligible(queryContext, query, finalExecBlockId, finalOutputDir)) {
+ hook.execute(context, queryContext, query, finalExecBlockId, finalOutputDir);
+ }
+ }
+ }
+ }
+
+ private class MaterializedResultHook implements QueryHook {
+
+ @Override
+ public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
+ Path finalOutputDir) {
+ Stage lastStage = query.getStage(finalExecBlockId);
+ NodeType type = lastStage.getBlock().getPlan().getType();
+ return type != NodeType.CREATE_TABLE && type != NodeType.INSERT;
+ }
+
+ @Override
+ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext,
+ Query query, ExecutionBlockId finalExecBlockId,
+ Path finalOutputDir) throws Exception {
+ Stage lastStage = query.getStage(finalExecBlockId);
+ TableMeta meta = lastStage.getTableMeta();
+
+ String nullChar = queryContext.get(SessionVars.NULL_CHAR);
+ meta.putOption(StorageConstants.TEXT_NULL, nullChar);
+
+ TableStats stats = lastStage.getResultStats();
+
+ TableDesc resultTableDesc =
+ new TableDesc(
+ query.getId().toString(),
+ lastStage.getSchema(),
+ meta,
+ finalOutputDir.toUri());
+ resultTableDesc.setExternal(true);
+
+ stats.setNumBytes(getTableVolume(query.systemConf, finalOutputDir));
+ resultTableDesc.setStats(stats);
+ query.setResultDesc(resultTableDesc);
+ }
+ }
+
+ private class CreateTableHook implements QueryHook {
+
+ @Override
+ public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
+ Path finalOutputDir) {
+ Stage lastStage = query.getStage(finalExecBlockId);
+ return lastStage.getBlock().getPlan().getType() == NodeType.CREATE_TABLE;
+ }
+
+ @Override
+ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext,
+ Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir) throws Exception {
+ CatalogService catalog = context.getWorkerContext().getCatalog();
+ Stage lastStage = query.getStage(finalExecBlockId);
+ TableStats stats = lastStage.getResultStats();
+
+ CreateTableNode createTableNode = (CreateTableNode) lastStage.getBlock().getPlan();
+ TableMeta meta = new TableMeta(createTableNode.getStorageType(), createTableNode.getOptions());
+
+ TableDesc tableDescTobeCreated =
+ new TableDesc(
+ createTableNode.getTableName(),
+ createTableNode.getTableSchema(),
+ meta,
+ finalOutputDir.toUri());
+ tableDescTobeCreated.setExternal(createTableNode.isExternal());
+
+ if (createTableNode.hasPartition()) {
+ tableDescTobeCreated.setPartitionMethod(createTableNode.getPartitionMethod());
+ }
+
+ stats.setNumBytes(getTableVolume(query.systemConf, finalOutputDir));
+ tableDescTobeCreated.setStats(stats);
+ query.setResultDesc(tableDescTobeCreated);
+
+ catalog.createTable(tableDescTobeCreated);
+ }
+ }
+
+ private class InsertTableHook implements QueryHook {
+
+ @Override
+ public boolean isEligible(QueryContext queryContext, Query query, ExecutionBlockId finalExecBlockId,
+ Path finalOutputDir) {
+ Stage lastStage = query.getStage(finalExecBlockId);
+ return lastStage.getBlock().getPlan().getType() == NodeType.INSERT;
+ }
+
+ @Override
+ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryContext,
+ Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir)
+ throws Exception {
+
+ CatalogService catalog = context.getWorkerContext().getCatalog();
+ Stage lastStage = query.getStage(finalExecBlockId);
+ TableMeta meta = lastStage.getTableMeta();
+ TableStats stats = lastStage.getResultStats();
+
+ InsertNode insertNode = (InsertNode) lastStage.getBlock().getPlan();
+
+ TableDesc finalTable;
+ if (insertNode.hasTargetTable()) {
+ String tableName = insertNode.getTableName();
+ finalTable = catalog.getTableDesc(tableName);
+ } else {
+ String tableName = query.getId().toString();
+ finalTable = new TableDesc(tableName, lastStage.getSchema(), meta, finalOutputDir.toUri());
+ }
+
+ long volume = getTableVolume(query.systemConf, finalOutputDir);
+ stats.setNumBytes(volume);
+ finalTable.setStats(stats);
+
+ if (insertNode.hasTargetTable()) {
+ UpdateTableStatsProto.Builder builder = UpdateTableStatsProto.newBuilder();
+ builder.setTableName(finalTable.getName());
+ builder.setStats(stats.getProto());
+
+ catalog.updateTableStats(builder.build());
+ }
+
+ query.setResultDesc(finalTable);
+ }
+ }
+ }
+
+ public static long getTableVolume(TajoConf systemConf, Path tablePath) throws IOException {
+ FileSystem fs = tablePath.getFileSystem(systemConf);
+ ContentSummary directorySummary = fs.getContentSummary(tablePath);
+ return directorySummary.getLength();
+ }
+
+ public static class StageCompletedTransition implements SingleArcTransition<Query, QueryEvent> {
+
+ private boolean hasNext(Query query) {
+ ExecutionBlockCursor cursor = query.getExecutionBlockCursor();
+ ExecutionBlock nextBlock = cursor.peek();
+ return !query.getPlan().isTerminal(nextBlock);
+ }
+
+ private void executeNextBlock(Query query) {
+ ExecutionBlockCursor cursor = query.getExecutionBlockCursor();
+ ExecutionBlock nextBlock = cursor.nextBlock();
+ Stage nextStage = new Stage(query.context, query.getPlan(), nextBlock);
+ nextStage.setPriority(query.priority--);
+ query.addStage(nextStage);
+ nextStage.handle(new StageEvent(nextStage.getId(), StageEventType.SQ_INIT));
+
+ LOG.info("Scheduling Stage:" + nextStage.getId());
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("Scheduling Stage's Priority: " + nextStage.getPriority());
+ LOG.debug("Scheduling Stage's Plan: \n" + nextStage.getBlock().getPlan());
+ }
+ }
+
+ @Override
+ public void transition(Query query, QueryEvent event) {
+ try {
+ query.completedStagesCount++;
+ StageCompletedEvent castEvent = (StageCompletedEvent) event;
+
+ if (castEvent.getState() == StageState.SUCCEEDED) {
+ query.successedStagesCount++;
+ } else if (castEvent.getState() == StageState.KILLED) {
+ query.killedStagesCount++;
+ } else if (castEvent.getState() == StageState.FAILED) {
+ query.failedStagesCount++;
+ } else if (castEvent.getState() == StageState.ERROR) {
+ query.erroredStagesCount++;
+ } else {
+ LOG.error(String.format("Invalid Stage (%s) State %s at %s",
+ castEvent.getExecutionBlockId().toString(), castEvent.getState().name(), query.getSynchronizedState().name()));
+ query.eventHandler.handle(new QueryEvent(event.getQueryId(), QueryEventType.INTERNAL_ERROR));
+ }
+
+ // if a stage is succeeded and a query is running
+ if (castEvent.getState() == StageState.SUCCEEDED && // latest stage succeeded
+ query.getSynchronizedState() == QueryState.QUERY_RUNNING && // current state is not in KILL_WAIT, FAILED, or ERROR.
+ hasNext(query)) { // there remains at least one stage.
+ query.getStage(castEvent.getExecutionBlockId()).waitingIntermediateReport();
+ executeNextBlock(query);
+ } else { // if a query is completed due to finished, kill, failure, or error
+ query.eventHandler.handle(new QueryCompletedEvent(castEvent.getExecutionBlockId(), castEvent.getState()));
+ }
+ } catch (Throwable t) {
+ LOG.error(t.getMessage(), t);
+ query.eventHandler.handle(new QueryEvent(event.getQueryId(), QueryEventType.INTERNAL_ERROR));
+ }
+ }
+ }
+
+ private static class DiagnosticsUpdateTransition implements SingleArcTransition<Query, QueryEvent> {
+ @Override
+ public void transition(Query query, QueryEvent event) {
+ query.addDiagnostic(((QueryDiagnosticsUpdateEvent) event).getDiagnosticUpdate());
+ }
+ }
+
+ private static class KillNewQueryTransition implements SingleArcTransition<Query, QueryEvent> {
+ @Override
+ public void transition(Query query, QueryEvent event) {
+ query.setFinishTime();
+ query.eventHandler.handle(new QueryMasterQueryCompletedEvent(query.getId()));
+ }
+ }
+
+ private static class KillAllStagesTransition implements SingleArcTransition<Query, QueryEvent> {
+ @Override
+ public void transition(Query query, QueryEvent event) {
+ synchronized (query.stages) {
+ for (Stage stage : query.stages.values()) {
+ query.eventHandler.handle(new StageEvent(stage.getId(), StageEventType.SQ_KILL));
+ }
+ }
+ }
+ }
+
+ private static class InternalErrorTransition implements SingleArcTransition<Query, QueryEvent> {
+
+ @Override
+ public void transition(Query query, QueryEvent event) {
+ query.setFinishTime();
+ query.eventHandler.handle(new QueryMasterQueryCompletedEvent(query.getId()));
+ }
+ }
+
+ @Override
+ public void handle(QueryEvent event) {
+ LOG.info("Processing " + event.getQueryId() + " of type " + event.getType());
+ try {
+ writeLock.lock();
+ QueryState oldState = getSynchronizedState();
+ try {
+ getStateMachine().doTransition(event.getType(), event);
+ queryState = getSynchronizedState();
+ } catch (InvalidStateTransitonException e) {
+ LOG.error("Can't handle this event at current state"
+ + ", type:" + event
+ + ", oldState:" + oldState.name()
+ + ", nextState:" + getSynchronizedState().name()
+ , e);
+ eventHandler.handle(new QueryEvent(this.id, QueryEventType.INTERNAL_ERROR));
+ }
+
+ //notify the eventhandler of state change
+ if (oldState != getSynchronizedState()) {
+ LOG.info(id + " Query Transitioned from " + oldState + " to " + getSynchronizedState());
+ }
+ }
+
+ finally {
+ writeLock.unlock();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryInProgress.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryInProgress.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryInProgress.java
new file mode 100644
index 0000000..bda2ec1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryInProgress.java
@@ -0,0 +1,301 @@
+/**
+ * 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.tajo.querymaster;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.ipc.QueryMasterProtocol;
+import org.apache.tajo.ipc.QueryMasterProtocol.QueryMasterProtocolService;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.ipc.TajoWorkerProtocol.QueryExecutionRequestProto;
+import org.apache.tajo.master.QueryInfo;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.rm.WorkerResourceManager;
+import org.apache.tajo.session.Session;
+import org.apache.tajo.plan.logical.LogicalRootNode;
+import org.apache.tajo.rpc.NettyClientBase;
+import org.apache.tajo.rpc.NullCallback;
+import org.apache.tajo.rpc.RpcConnectionPool;
+import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
+import org.apache.tajo.util.NetUtils;
+
+import java.net.InetSocketAddress;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.tajo.ipc.TajoMasterProtocol.WorkerAllocatedResource;
+
+public class QueryInProgress extends CompositeService {
+ private static final Log LOG = LogFactory.getLog(QueryInProgress.class.getName());
+
+ private QueryId queryId;
+
+ private Session session;
+
+ private AsyncDispatcher dispatcher;
+
+ private LogicalRootNode plan;
+
+ private AtomicBoolean querySubmitted = new AtomicBoolean(false);
+
+ private AtomicBoolean stopped = new AtomicBoolean(false);
+
+ private QueryInfo queryInfo;
+
+ private final TajoMaster.MasterContext masterContext;
+
+ private NettyClientBase queryMasterRpc;
+
+ private QueryMasterProtocolService queryMasterRpcClient;
+
+ public QueryInProgress(
+ TajoMaster.MasterContext masterContext,
+ Session session,
+ QueryContext queryContext,
+ QueryId queryId, String sql, String jsonExpr, LogicalRootNode plan) {
+ super(QueryInProgress.class.getName());
+ this.masterContext = masterContext;
+ this.session = session;
+ this.queryId = queryId;
+ this.plan = plan;
+
+ queryInfo = new QueryInfo(queryId, queryContext, sql, jsonExpr);
+ queryInfo.setStartTime(System.currentTimeMillis());
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ dispatcher = new AsyncDispatcher();
+ this.addService(dispatcher);
+
+ dispatcher.register(QueryJobEvent.Type.class, new QueryInProgressEventHandler());
+ super.init(conf);
+ }
+
+ public synchronized void kill() {
+ if(queryMasterRpcClient != null){
+ queryMasterRpcClient.killQuery(null, queryId.getProto(), NullCallback.get());
+ }
+ }
+
+ @Override
+ public void stop() {
+ if(stopped.getAndSet(true)) {
+ return;
+ }
+
+ LOG.info("=========================================================");
+ LOG.info("Stop query:" + queryId);
+
+ masterContext.getResourceManager().stopQueryMaster(queryId);
+
+ long startTime = System.currentTimeMillis();
+ while(true) {
+ try {
+ if(masterContext.getResourceManager().isQueryMasterStopped(queryId)) {
+ LOG.info(queryId + " QueryMaster stopped");
+ break;
+ }
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ break;
+ }
+
+ try {
+ synchronized (this){
+ wait(100);
+ }
+ } catch (InterruptedException e) {
+ break;
+ }
+ if(System.currentTimeMillis() - startTime > 60 * 1000) {
+ LOG.warn("Failed to stop QueryMaster:" + queryId);
+ break;
+ }
+ }
+
+ if(queryMasterRpc != null) {
+ RpcConnectionPool.getPool(masterContext.getConf()).closeConnection(queryMasterRpc);
+ }
+
+ masterContext.getHistoryWriter().appendHistory(queryInfo);
+ super.stop();
+ }
+
+ @Override
+ public void start() {
+ super.start();
+ }
+
+ public EventHandler getEventHandler() {
+ return dispatcher.getEventHandler();
+ }
+
+
+
+ public boolean startQueryMaster() {
+ try {
+ LOG.info("Initializing QueryInProgress for QueryID=" + queryId);
+ WorkerResourceManager resourceManager = masterContext.getResourceManager();
+ WorkerAllocatedResource resource = resourceManager.allocateQueryMaster(this);
+
+ // if no resource to allocate a query master
+ if(resource == null) {
+ LOG.info("No Available Resources for QueryMaster");
+ return false;
+ }
+
+ queryInfo.setQueryMaster(resource.getConnectionInfo().getHost());
+ queryInfo.setQueryMasterPort(resource.getConnectionInfo().getQueryMasterPort());
+ queryInfo.setQueryMasterclientPort(resource.getConnectionInfo().getClientPort());
+ queryInfo.setQueryMasterInfoPort(resource.getConnectionInfo().getHttpInfoPort());
+
+ getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_MASTER_START, queryInfo));
+
+ return true;
+ } catch (Exception e) {
+ catchException(e);
+ return false;
+ }
+ }
+
+ class QueryInProgressEventHandler implements EventHandler<QueryJobEvent> {
+ @Override
+ public void handle(QueryJobEvent queryJobEvent) {
+ if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_HEARTBEAT) {
+ heartbeat(queryJobEvent.getQueryInfo());
+ } else if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_MASTER_START) {
+ QueryInProgress queryInProgress = masterContext.getQueryJobManager().getQueryInProgress(queryId);
+ queryInProgress.getEventHandler().handle(
+ new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_START, queryInProgress.getQueryInfo()));
+ } else if(queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_START) {
+ submmitQueryToMaster();
+ } else if (queryJobEvent.getType() == QueryJobEvent.Type.QUERY_JOB_KILL) {
+ kill();
+ }
+ }
+ }
+
+ private void connectQueryMaster() throws Exception {
+ InetSocketAddress addr = NetUtils.createSocketAddr(queryInfo.getQueryMasterHost(), queryInfo.getQueryMasterPort());
+ LOG.info("Connect to QueryMaster:" + addr);
+ queryMasterRpc =
+ RpcConnectionPool.getPool(masterContext.getConf()).getConnection(addr, QueryMasterProtocol.class, true);
+ queryMasterRpcClient = queryMasterRpc.getStub();
+ }
+
+ private synchronized void submmitQueryToMaster() {
+ if(querySubmitted.get()) {
+ return;
+ }
+
+ try {
+ if(queryMasterRpcClient == null) {
+ connectQueryMaster();
+ }
+ if(queryMasterRpcClient == null) {
+ LOG.info("No QueryMaster conneciton info.");
+ //TODO wait
+ return;
+ }
+ LOG.info("Call executeQuery to :" +
+ queryInfo.getQueryMasterHost() + ":" + queryInfo.getQueryMasterPort() + "," + queryId);
+
+ QueryExecutionRequestProto.Builder builder = TajoWorkerProtocol.QueryExecutionRequestProto.newBuilder();
+ builder.setQueryId(queryId.getProto())
+ .setQueryContext(queryInfo.getQueryContext().getProto())
+ .setSession(session.getProto())
+ .setExprInJson(PrimitiveProtos.StringProto.newBuilder().setValue(queryInfo.getJsonExpr()))
+ .setLogicalPlanJson(PrimitiveProtos.StringProto.newBuilder().setValue(plan.toJson()).build());
+
+ queryMasterRpcClient.executeQuery(null, builder.build(), NullCallback.get());
+ querySubmitted.set(true);
+ } catch (Exception e) {
+ LOG.error(e.getMessage(), e);
+ }
+ }
+
+ public void catchException(Exception e) {
+ LOG.error(e.getMessage(), e);
+ queryInfo.setQueryState(TajoProtos.QueryState.QUERY_FAILED);
+ queryInfo.setLastMessage(StringUtils.stringifyException(e));
+ }
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public QueryInfo getQueryInfo() {
+ return this.queryInfo;
+ }
+
+ public boolean isStarted() {
+ return !stopped.get() && this.querySubmitted.get();
+ }
+
+ private void heartbeat(QueryInfo queryInfo) {
+ LOG.info("Received QueryMaster heartbeat:" + queryInfo);
+
+ // to avoid partial update by different heartbeats
+ synchronized (this.queryInfo) {
+
+ // terminal state will let client to retrieve a query result
+ // So, we must set the query result before changing query state
+ if (isFinishState(queryInfo.getQueryState())) {
+ if (queryInfo.hasResultdesc()) {
+ this.queryInfo.setResultDesc(queryInfo.getResultDesc());
+ }
+ }
+
+ this.queryInfo.setQueryState(queryInfo.getQueryState());
+ this.queryInfo.setProgress(queryInfo.getProgress());
+ this.queryInfo.setFinishTime(queryInfo.getFinishTime());
+
+ // Update diagnosis message
+ if (queryInfo.getLastMessage() != null && !queryInfo.getLastMessage().isEmpty()) {
+ this.queryInfo.setLastMessage(queryInfo.getLastMessage());
+ LOG.info(queryId + queryInfo.getLastMessage());
+ }
+
+ // if any error occurs, print outs the error message
+ if (this.queryInfo.getQueryState() == TajoProtos.QueryState.QUERY_FAILED) {
+ LOG.warn(queryId + " failed, " + queryInfo.getLastMessage());
+ }
+
+
+ if (isFinishState(this.queryInfo.getQueryState())) {
+ masterContext.getQueryJobManager().getEventHandler().handle(
+ new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_STOP, this.queryInfo));
+ }
+ }
+ }
+
+ private boolean isFinishState(TajoProtos.QueryState state) {
+ return state == TajoProtos.QueryState.QUERY_FAILED ||
+ state == TajoProtos.QueryState.QUERY_KILLED ||
+ state == TajoProtos.QueryState.QUERY_SUCCEEDED;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryJobEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryJobEvent.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryJobEvent.java
new file mode 100644
index 0000000..1a1f2ff
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/QueryJobEvent.java
@@ -0,0 +1,46 @@
+/**
+ * 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.tajo.querymaster;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+import org.apache.tajo.master.QueryInfo;
+
+public class QueryJobEvent extends AbstractEvent<QueryJobEvent.Type> {
+ private QueryInfo queryInfo;
+
+ public QueryJobEvent(Type type, QueryInfo queryInfo) {
+ super(type);
+
+ this.queryInfo = queryInfo;
+ }
+
+ public QueryInfo getQueryInfo() {
+ return this.queryInfo;
+ }
+
+ public enum Type {
+ QUERY_JOB_START,
+ QUERY_JOB_HEARTBEAT,
+ QUERY_JOB_FINISH,
+ QUERY_JOB_STOP,
+ QUERY_MASTER_START,
+ QUERY_MASTER_STOP,
+ QUERY_JOB_KILL
+ }
+}
[10/16] tajo git commit: TAJO-1288: Refactoring
org.apache.tajo.master package.
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
deleted file mode 100644
index 367897d..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
+++ /dev/null
@@ -1,1251 +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.tajo.master.querymaster;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.Path;
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.SessionVars;
-import org.apache.tajo.algebra.JoinType;
-import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.catalog.statistics.StatisticsUtil;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
-import org.apache.tajo.engine.planner.RangePartitionAlgorithm;
-import org.apache.tajo.engine.planner.UniformRangePartition;
-import org.apache.tajo.engine.planner.enforce.Enforcer;
-import org.apache.tajo.engine.planner.global.DataChannel;
-import org.apache.tajo.engine.planner.global.ExecutionBlock;
-import org.apache.tajo.engine.planner.global.GlobalPlanner;
-import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.engine.utils.TupleUtil;
-import org.apache.tajo.exception.InternalException;
-import org.apache.tajo.ipc.TajoWorkerProtocol.DistinctGroupbyEnforcer.MultipleAggregationStage;
-import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty;
-import org.apache.tajo.master.TaskSchedulerContext;
-import org.apache.tajo.master.querymaster.Task.IntermediateEntry;
-import org.apache.tajo.plan.logical.SortNode.SortPurpose;
-import org.apache.tajo.plan.util.PlannerUtil;
-import org.apache.tajo.plan.PlanningException;
-import org.apache.tajo.plan.logical.*;
-import org.apache.tajo.storage.FileStorageManager;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.RowStoreUtil;
-import org.apache.tajo.storage.TupleRange;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.util.Pair;
-import org.apache.tajo.unit.StorageUnit;
-import org.apache.tajo.util.TUtil;
-import org.apache.tajo.util.TajoIdUtils;
-import org.apache.tajo.worker.FetchImpl;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.math.BigInteger;
-import java.net.URI;
-import java.util.*;
-import java.util.Map.Entry;
-
-import static org.apache.tajo.plan.serder.PlanProto.ShuffleType;
-import static org.apache.tajo.plan.serder.PlanProto.ShuffleType.*;
-
-/**
- * Repartitioner creates non-leaf tasks and shuffles intermediate data.
- * It supports two repartition methods, such as hash and range repartition.
- */
-public class Repartitioner {
- private static final Log LOG = LogFactory.getLog(Repartitioner.class);
-
- private final static int HTTP_REQUEST_MAXIMUM_LENGTH = 1900;
- private final static String UNKNOWN_HOST = "unknown";
-
- public static void scheduleFragmentsForJoinQuery(TaskSchedulerContext schedulerContext, Stage stage)
- throws IOException {
- MasterPlan masterPlan = stage.getMasterPlan();
- ExecutionBlock execBlock = stage.getBlock();
- QueryMasterTask.QueryMasterTaskContext masterContext = stage.getContext();
-
- ScanNode[] scans = execBlock.getScanNodes();
-
- Path tablePath;
- Fragment[] fragments = new Fragment[scans.length];
- long[] stats = new long[scans.length];
-
- // initialize variables from the child operators
- for (int i = 0; i < scans.length; i++) {
- TableDesc tableDesc = masterContext.getTableDescMap().get(scans[i].getCanonicalName());
- if (tableDesc == null) { // if it is a real table stored on storage
- FileStorageManager storageManager =
- (FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf());
-
- tablePath = storageManager.getTablePath(scans[i].getTableName());
- if (execBlock.getUnionScanMap() != null && !execBlock.getUnionScanMap().isEmpty()) {
- for (Map.Entry<ExecutionBlockId, ExecutionBlockId> unionScanEntry: execBlock.getUnionScanMap().entrySet()) {
- ExecutionBlockId originScanEbId = unionScanEntry.getKey();
- stats[i] += masterContext.getStage(originScanEbId).getResultStats().getNumBytes();
- }
- } else {
- ExecutionBlockId scanEBId = TajoIdUtils.createExecutionBlockId(scans[i].getTableName());
- stats[i] = masterContext.getStage(scanEBId).getResultStats().getNumBytes();
- }
- fragments[i] = new FileFragment(scans[i].getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
- } else {
- try {
- stats[i] = GlobalPlanner.computeDescendentVolume(scans[i]);
- } catch (PlanningException e) {
- throw new IOException(e);
- }
-
- StorageManager storageManager =
- StorageManager.getStorageManager(stage.getContext().getConf(), tableDesc.getMeta().getStoreType());
-
- // if table has no data, storageManager will return empty FileFragment.
- // So, we need to handle FileFragment by its size.
- // If we don't check its size, it can cause IndexOutOfBoundsException.
- List<Fragment> fileFragments = storageManager.getSplits(scans[i].getCanonicalName(), tableDesc);
- if (fileFragments.size() > 0) {
- fragments[i] = fileFragments.get(0);
- } else {
- fragments[i] = new FileFragment(scans[i].getCanonicalName(), new Path(tableDesc.getPath()), 0, 0, new String[]{UNKNOWN_HOST});
- }
- }
- }
-
- // If one of inner join tables has no input data, it means that this execution block has no result row.
- JoinNode joinNode = PlannerUtil.findMostBottomNode(execBlock.getPlan(), NodeType.JOIN);
- if (joinNode != null) {
- if ( (joinNode.getJoinType() == JoinType.INNER)) {
- LogicalNode leftNode = joinNode.getLeftChild();
- LogicalNode rightNode = joinNode.getRightChild();
- for (int i = 0; i < stats.length; i++) {
- if (scans[i].getPID() == leftNode.getPID() || scans[i].getPID() == rightNode.getPID()) {
- if (stats[i] == 0) {
- LOG.info(scans[i] + " 's input data is zero. Inner join's result is empty.");
- return;
- }
- }
- }
- }
- }
-
- // If node is outer join and a preserved relation is empty, it should return zero rows.
- joinNode = PlannerUtil.findTopNode(execBlock.getPlan(), NodeType.JOIN);
- if (joinNode != null) {
- // If all stats are zero, return
- boolean isEmptyAllJoinTables = true;
- for (int i = 0; i < stats.length; i++) {
- if (stats[i] > 0) {
- isEmptyAllJoinTables = false;
- break;
- }
- }
- if (isEmptyAllJoinTables) {
- LOG.info("All input join tables are empty.");
- return;
- }
-
- // find left top scan node
- ScanNode leftScanNode = PlannerUtil.findTopNode(joinNode.getLeftChild(), NodeType.SCAN);
- ScanNode rightScanNode = PlannerUtil.findTopNode(joinNode.getRightChild(), NodeType.SCAN);
-
- long leftStats = -1;
- long rightStats = -1;
- if (stats.length == 2) {
- for (int i = 0; i < stats.length; i++) {
- if (scans[i].equals(leftScanNode)) {
- leftStats = stats[i];
- } else if (scans[i].equals(rightScanNode)) {
- rightStats = stats[i];
- }
- }
- if (joinNode.getJoinType() == JoinType.LEFT_OUTER) {
- if (leftStats == 0) {
- return;
- }
- }
- if (joinNode.getJoinType() == JoinType.RIGHT_OUTER) {
- if (rightStats == 0) {
- return;
- }
- }
- }
- }
-
- // Assigning either fragments or fetch urls to query units
- boolean isAllBroadcastTable = true;
- for (int i = 0; i < scans.length; i++) {
- if (!execBlock.isBroadcastTable(scans[i].getCanonicalName())) {
- isAllBroadcastTable = false;
- break;
- }
- }
-
-
- if (isAllBroadcastTable) { // if all relations of this EB are broadcasted
- // set largest table to normal mode
- long maxStats = Long.MIN_VALUE;
- int maxStatsScanIdx = -1;
- for (int i = 0; i < scans.length; i++) {
- // finding largest table.
- // If stats == 0, can't be base table.
- if (stats[i] > 0 && stats[i] > maxStats) {
- maxStats = stats[i];
- maxStatsScanIdx = i;
- }
- }
- if (maxStatsScanIdx == -1) {
- maxStatsScanIdx = 0;
- }
- int baseScanIdx = maxStatsScanIdx;
- scans[baseScanIdx].setBroadcastTable(false);
- execBlock.removeBroadcastTable(scans[baseScanIdx].getCanonicalName());
- LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join with all tables, base_table=%s, base_volume=%d",
- scans[baseScanIdx].getCanonicalName(), stats[baseScanIdx]));
- scheduleLeafTasksWithBroadcastTable(schedulerContext, stage, baseScanIdx, fragments);
- } else if (!execBlock.getBroadcastTables().isEmpty()) { // If some relations of this EB are broadcasted
- boolean hasNonLeafNode = false;
- List<Integer> largeScanIndexList = new ArrayList<Integer>();
- List<Integer> broadcastIndexList = new ArrayList<Integer>();
- String nonLeafScanNames = "";
- String namePrefix = "";
- long maxStats = Long.MIN_VALUE;
- int maxStatsScanIdx = -1;
- for (int i = 0; i < scans.length; i++) {
- if (scans[i].getTableDesc().getMeta().getStoreType() == StoreType.RAW) {
- // Intermediate data scan
- hasNonLeafNode = true;
- largeScanIndexList.add(i);
- nonLeafScanNames += namePrefix + scans[i].getCanonicalName();
- namePrefix = ",";
- }
- if (execBlock.isBroadcastTable(scans[i].getCanonicalName())) {
- broadcastIndexList.add(i);
- } else {
- // finding largest table.
- if (stats[i] > 0 && stats[i] > maxStats) {
- maxStats = stats[i];
- maxStatsScanIdx = i;
- }
- }
- }
- if (maxStatsScanIdx == -1) {
- maxStatsScanIdx = 0;
- }
-
- if (!hasNonLeafNode) {
- if (largeScanIndexList.size() > 1) {
- String largeTableNames = "";
- for (Integer eachId : largeScanIndexList) {
- largeTableNames += scans[eachId].getTableName() + ",";
- }
- throw new IOException("Broadcast join with leaf node should have only one large table, " +
- "but " + largeScanIndexList.size() + ", tables=" + largeTableNames);
- }
- int baseScanIdx = largeScanIndexList.isEmpty() ? maxStatsScanIdx : largeScanIndexList.get(0);
- LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join, base_table=%s, base_volume=%d",
- scans[baseScanIdx].getCanonicalName(), stats[baseScanIdx]));
- scheduleLeafTasksWithBroadcastTable(schedulerContext, stage, baseScanIdx, fragments);
- } else {
- if (largeScanIndexList.size() > 2) {
- throw new IOException("Symmetric Repartition Join should have two scan node, but " + nonLeafScanNames);
- }
-
- //select intermediate scan and stats
- ScanNode[] intermediateScans = new ScanNode[largeScanIndexList.size()];
- long[] intermediateScanStats = new long[largeScanIndexList.size()];
- Fragment[] intermediateFragments = new Fragment[largeScanIndexList.size()];
- int index = 0;
- for (Integer eachIdx : largeScanIndexList) {
- intermediateScans[index] = scans[eachIdx];
- intermediateScanStats[index] = stats[eachIdx];
- intermediateFragments[index++] = fragments[eachIdx];
- }
- Fragment[] broadcastFragments = new Fragment[broadcastIndexList.size()];
- ScanNode[] broadcastScans = new ScanNode[broadcastIndexList.size()];
- index = 0;
- for (Integer eachIdx : broadcastIndexList) {
- scans[eachIdx].setBroadcastTable(true);
- broadcastScans[index] = scans[eachIdx];
- broadcastFragments[index] = fragments[eachIdx];
- index++;
- }
- LOG.info(String.format("[Distributed Join Strategy] : Broadcast Join, join_node=%s", nonLeafScanNames));
- scheduleSymmetricRepartitionJoin(masterContext, schedulerContext, stage,
- intermediateScans, intermediateScanStats, intermediateFragments, broadcastScans, broadcastFragments);
- }
- } else {
- LOG.info("[Distributed Join Strategy] : Symmetric Repartition Join");
- scheduleSymmetricRepartitionJoin(masterContext, schedulerContext, stage, scans, stats, fragments, null, null);
- }
- }
-
- /**
- * Scheduling in tech case of Symmetric Repartition Join
- * @param masterContext
- * @param schedulerContext
- * @param stage
- * @param scans
- * @param stats
- * @param fragments
- * @throws IOException
- */
- private static void scheduleSymmetricRepartitionJoin(QueryMasterTask.QueryMasterTaskContext masterContext,
- TaskSchedulerContext schedulerContext,
- Stage stage,
- ScanNode[] scans,
- long[] stats,
- Fragment[] fragments,
- ScanNode[] broadcastScans,
- Fragment[] broadcastFragments) throws IOException {
- MasterPlan masterPlan = stage.getMasterPlan();
- ExecutionBlock execBlock = stage.getBlock();
- // The hash map is modeling as follows:
- // <Part Id, <EbId, List<Intermediate Data>>>
- Map<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>> hashEntries =
- new HashMap<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>>();
-
- // Grouping IntermediateData by a partition key and a table name
- List<ExecutionBlock> childBlocks = masterPlan.getChilds(stage.getId());
-
- // In the case of join with union, there is one ScanNode for union.
- Map<ExecutionBlockId, ExecutionBlockId> unionScanMap = execBlock.getUnionScanMap();
- for (ExecutionBlock childBlock : childBlocks) {
- ExecutionBlockId scanEbId = unionScanMap.get(childBlock.getId());
- if (scanEbId == null) {
- scanEbId = childBlock.getId();
- }
- Stage childExecSM = stage.getContext().getStage(childBlock.getId());
-
- if (childExecSM.getHashShuffleIntermediateEntries() != null &&
- !childExecSM.getHashShuffleIntermediateEntries().isEmpty()) {
- for (IntermediateEntry intermediateEntry: childExecSM.getHashShuffleIntermediateEntries()) {
- intermediateEntry.setEbId(childBlock.getId());
- if (hashEntries.containsKey(intermediateEntry.getPartId())) {
- Map<ExecutionBlockId, List<IntermediateEntry>> tbNameToInterm =
- hashEntries.get(intermediateEntry.getPartId());
-
- if (tbNameToInterm.containsKey(scanEbId)) {
- tbNameToInterm.get(scanEbId).add(intermediateEntry);
- } else {
- tbNameToInterm.put(scanEbId, TUtil.newList(intermediateEntry));
- }
- } else {
- Map<ExecutionBlockId, List<IntermediateEntry>> tbNameToInterm =
- new HashMap<ExecutionBlockId, List<IntermediateEntry>>();
- tbNameToInterm.put(scanEbId, TUtil.newList(intermediateEntry));
- hashEntries.put(intermediateEntry.getPartId(), tbNameToInterm);
- }
- }
- } else {
- //if no intermidatedata(empty table), make empty entry
- int emptyPartitionId = 0;
- if (hashEntries.containsKey(emptyPartitionId)) {
- Map<ExecutionBlockId, List<IntermediateEntry>> tbNameToInterm = hashEntries.get(emptyPartitionId);
- if (tbNameToInterm.containsKey(scanEbId))
- tbNameToInterm.get(scanEbId).addAll(new ArrayList<IntermediateEntry>());
- else
- tbNameToInterm.put(scanEbId, new ArrayList<IntermediateEntry>());
- } else {
- Map<ExecutionBlockId, List<IntermediateEntry>> tbNameToInterm =
- new HashMap<ExecutionBlockId, List<IntermediateEntry>>();
- tbNameToInterm.put(scanEbId, new ArrayList<IntermediateEntry>());
- hashEntries.put(emptyPartitionId, tbNameToInterm);
- }
- }
- }
-
- // hashEntries can be zero if there are no input data.
- // In the case, it will cause the zero divided exception.
- // it avoids this problem.
- int[] avgSize = new int[2];
- avgSize[0] = hashEntries.size() == 0 ? 0 : (int) (stats[0] / hashEntries.size());
- avgSize[1] = hashEntries.size() == 0 ? 0 : (int) (stats[1] / hashEntries.size());
- int bothFetchSize = avgSize[0] + avgSize[1];
-
- // Getting the desire number of join tasks according to the volumn
- // of a larger table
- int largerIdx = stats[0] >= stats[1] ? 0 : 1;
- int desireJoinTaskVolumn = stage.getMasterPlan().getContext().getInt(SessionVars.JOIN_TASK_INPUT_SIZE);
-
- // calculate the number of tasks according to the data size
- int mb = (int) Math.ceil((double) stats[largerIdx] / 1048576);
- LOG.info("Larger intermediate data is approximately " + mb + " MB");
- // determine the number of task per 64MB
- int maxTaskNum = (int) Math.ceil((double) mb / desireJoinTaskVolumn);
- LOG.info("The calculated number of tasks is " + maxTaskNum);
- LOG.info("The number of total shuffle keys is " + hashEntries.size());
- // the number of join tasks cannot be larger than the number of
- // distinct partition ids.
- int joinTaskNum = Math.min(maxTaskNum, hashEntries.size());
- LOG.info("The determined number of join tasks is " + joinTaskNum);
-
- List<Fragment> rightFragments = new ArrayList<Fragment>();
- rightFragments.add(fragments[1]);
-
- if (broadcastFragments != null) {
- //In this phase a ScanNode has a single fragment.
- //If there are more than one data files, that files should be added to fragments or partition path
- for (ScanNode eachScan: broadcastScans) {
- Path[] partitionScanPaths = null;
- TableDesc tableDesc = masterContext.getTableDescMap().get(eachScan.getCanonicalName());
- if (eachScan.getType() == NodeType.PARTITIONS_SCAN) {
- FileStorageManager storageManager =
- (FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf());
-
- PartitionedTableScanNode partitionScan = (PartitionedTableScanNode)eachScan;
- partitionScanPaths = partitionScan.getInputPaths();
- // set null to inputPaths in getFragmentsFromPartitionedTable()
- getFragmentsFromPartitionedTable(storageManager, eachScan, tableDesc);
- partitionScan.setInputPaths(partitionScanPaths);
- } else {
- StorageManager storageManager = StorageManager.getStorageManager(stage.getContext().getConf(),
- tableDesc.getMeta().getStoreType());
- Collection<Fragment> scanFragments = storageManager.getSplits(eachScan.getCanonicalName(),
- tableDesc, eachScan);
- if (scanFragments != null) {
- rightFragments.addAll(scanFragments);
- }
- }
- }
- }
- Stage.scheduleFragment(stage, fragments[0], rightFragments);
-
- // Assign partitions to tasks in a round robin manner.
- for (Entry<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>> entry
- : hashEntries.entrySet()) {
- addJoinShuffle(stage, entry.getKey(), entry.getValue());
- }
-
- schedulerContext.setTaskSize((int) Math.ceil((double) bothFetchSize / joinTaskNum));
- schedulerContext.setEstimatedTaskNum(joinTaskNum);
- }
-
- /**
- * merge intermediate entry by ebid, pullhost
- * @param hashEntries
- * @return
- */
- public static Map<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>> mergeIntermediateByPullHost(
- Map<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>> hashEntries) {
- Map<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>> mergedHashEntries =
- new HashMap<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>>();
-
- for(Entry<Integer, Map<ExecutionBlockId, List<IntermediateEntry>>> entry: hashEntries.entrySet()) {
- Integer partId = entry.getKey();
- for (Entry<ExecutionBlockId, List<IntermediateEntry>> partEntry: entry.getValue().entrySet()) {
- List<IntermediateEntry> intermediateList = partEntry.getValue();
- if (intermediateList == null || intermediateList.isEmpty()) {
- continue;
- }
- ExecutionBlockId ebId = partEntry.getKey();
- // EBID + PullHost -> IntermediateEntry
- // In the case of union partEntry.getKey() return's delegated EBID.
- // Intermediate entries are merged by real EBID.
- Map<String, IntermediateEntry> ebMerged = new HashMap<String, IntermediateEntry>();
-
- for (IntermediateEntry eachIntermediate: intermediateList) {
- String ebMergedKey = eachIntermediate.getEbId().toString() + eachIntermediate.getPullHost().getPullAddress();
- IntermediateEntry intermediateEntryPerPullHost = ebMerged.get(ebMergedKey);
- if (intermediateEntryPerPullHost == null) {
- intermediateEntryPerPullHost = new IntermediateEntry(-1, -1, partId, eachIntermediate.getPullHost());
- intermediateEntryPerPullHost.setEbId(eachIntermediate.getEbId());
- ebMerged.put(ebMergedKey, intermediateEntryPerPullHost);
- }
- intermediateEntryPerPullHost.setVolume(intermediateEntryPerPullHost.getVolume() + eachIntermediate.getVolume());
- }
-
- List<IntermediateEntry> ebIntermediateEntries = new ArrayList<IntermediateEntry>(ebMerged.values());
-
- Map<ExecutionBlockId, List<IntermediateEntry>> mergedPartEntries = mergedHashEntries.get(partId);
- if (mergedPartEntries == null) {
- mergedPartEntries = new HashMap<ExecutionBlockId, List<IntermediateEntry>>();
- mergedHashEntries.put(partId, mergedPartEntries);
- }
- mergedPartEntries.put(ebId, ebIntermediateEntries);
- }
- }
- return mergedHashEntries;
- }
-
- /**
- * It creates a number of fragments for all partitions.
- */
- public static List<Fragment> getFragmentsFromPartitionedTable(FileStorageManager sm,
- ScanNode scan,
- TableDesc table) throws IOException {
- List<Fragment> fragments = Lists.newArrayList();
- PartitionedTableScanNode partitionsScan = (PartitionedTableScanNode) scan;
- fragments.addAll(sm.getSplits(
- scan.getCanonicalName(), table.getMeta(), table.getSchema(), partitionsScan.getInputPaths()));
- partitionsScan.setInputPaths(null);
- return fragments;
- }
-
- private static void scheduleLeafTasksWithBroadcastTable(TaskSchedulerContext schedulerContext, Stage stage,
- int baseScanId, Fragment[] fragments) throws IOException {
- ExecutionBlock execBlock = stage.getBlock();
- ScanNode[] scans = execBlock.getScanNodes();
-
- for (int i = 0; i < scans.length; i++) {
- if (i != baseScanId) {
- scans[i].setBroadcastTable(true);
- }
- }
-
- // Large table(baseScan)
- // -> add all fragment to baseFragments
- // -> each fragment is assigned to a Task by DefaultTaskScheduler.handle()
- // Broadcast table
- // all fragments or paths assigned every Large table's scan task.
- // -> PARTITIONS_SCAN
- // . add all partition paths to node's inputPaths variable
- // -> SCAN
- // . add all fragments to broadcastFragments
- Collection<Fragment> baseFragments = null;
- List<Fragment> broadcastFragments = new ArrayList<Fragment>();
- for (int i = 0; i < scans.length; i++) {
- ScanNode scan = scans[i];
- TableDesc desc = stage.getContext().getTableDescMap().get(scan.getCanonicalName());
- TableMeta meta = desc.getMeta();
-
- Collection<Fragment> scanFragments;
- Path[] partitionScanPaths = null;
- if (scan.getType() == NodeType.PARTITIONS_SCAN) {
- PartitionedTableScanNode partitionScan = (PartitionedTableScanNode)scan;
- partitionScanPaths = partitionScan.getInputPaths();
- // set null to inputPaths in getFragmentsFromPartitionedTable()
- FileStorageManager storageManager =
- (FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf());
- scanFragments = getFragmentsFromPartitionedTable(storageManager, scan, desc);
- } else {
- StorageManager storageManager =
- StorageManager.getStorageManager(stage.getContext().getConf(), desc.getMeta().getStoreType());
-
- scanFragments = storageManager.getSplits(scan.getCanonicalName(), desc, scan);
- }
-
- if (scanFragments != null) {
- if (i == baseScanId) {
- baseFragments = scanFragments;
- } else {
- if (scan.getType() == NodeType.PARTITIONS_SCAN) {
- PartitionedTableScanNode partitionScan = (PartitionedTableScanNode)scan;
- // PhisicalPlanner make PartitionMergeScanExec when table is boradcast table and inputpaths is not empty
- partitionScan.setInputPaths(partitionScanPaths);
- } else {
- broadcastFragments.addAll(scanFragments);
- }
- }
- }
- }
-
- if (baseFragments == null) {
- throw new IOException("No fragments for " + scans[baseScanId].getTableName());
- }
-
- Stage.scheduleFragments(stage, baseFragments, broadcastFragments);
- schedulerContext.setEstimatedTaskNum(baseFragments.size());
- }
-
- private static void addJoinShuffle(Stage stage, int partitionId,
- Map<ExecutionBlockId, List<IntermediateEntry>> grouppedPartitions) {
- Map<String, List<FetchImpl>> fetches = new HashMap<String, List<FetchImpl>>();
- for (ExecutionBlock execBlock : stage.getMasterPlan().getChilds(stage.getId())) {
- if (grouppedPartitions.containsKey(execBlock.getId())) {
- Collection<FetchImpl> requests = mergeShuffleRequest(partitionId, HASH_SHUFFLE,
- grouppedPartitions.get(execBlock.getId()));
- fetches.put(execBlock.getId().toString(), Lists.newArrayList(requests));
- }
- }
-
- if (fetches.isEmpty()) {
- LOG.info(stage.getId() + "'s " + partitionId + " partition has empty result.");
- return;
- }
- Stage.scheduleFetches(stage, fetches);
- }
-
- /**
- * This method merges the partition request associated with the pullserver's address.
- * It reduces the number of TCP connections.
- *
- * @return key: pullserver's address, value: a list of requests
- */
- private static Collection<FetchImpl> mergeShuffleRequest(int partitionId,
- ShuffleType type,
- List<IntermediateEntry> partitions) {
- // ebId + pullhost -> FetchImmpl
- Map<String, FetchImpl> mergedPartitions = new HashMap<String, FetchImpl>();
-
- for (IntermediateEntry partition : partitions) {
- String mergedKey = partition.getEbId().toString() + "," + partition.getPullHost();
-
- if (mergedPartitions.containsKey(mergedKey)) {
- FetchImpl fetch = mergedPartitions.get(mergedKey);
- fetch.addPart(partition.getTaskId(), partition.getAttemptId());
- } else {
- // In some cases like union each IntermediateEntry has different EBID.
- FetchImpl fetch = new FetchImpl(partition.getPullHost(), type, partition.getEbId(), partitionId);
- fetch.addPart(partition.getTaskId(), partition.getAttemptId());
- mergedPartitions.put(mergedKey, fetch);
- }
- }
- return mergedPartitions.values();
- }
-
- public static void scheduleFragmentsForNonLeafTasks(TaskSchedulerContext schedulerContext,
- MasterPlan masterPlan, Stage stage, int maxNum)
- throws IOException {
- DataChannel channel = masterPlan.getIncomingChannels(stage.getBlock().getId()).get(0);
- if (channel.getShuffleType() == HASH_SHUFFLE
- || channel.getShuffleType() == SCATTERED_HASH_SHUFFLE) {
- scheduleHashShuffledFetches(schedulerContext, masterPlan, stage, channel, maxNum);
- } else if (channel.getShuffleType() == RANGE_SHUFFLE) {
- scheduleRangeShuffledFetches(schedulerContext, masterPlan, stage, channel, maxNum);
- } else {
- throw new InternalException("Cannot support partition type");
- }
- }
-
- private static TableStats computeChildBlocksStats(QueryMasterTask.QueryMasterTaskContext context, MasterPlan masterPlan,
- ExecutionBlockId parentBlockId) {
- List<TableStats> tableStatses = new ArrayList<TableStats>();
- List<ExecutionBlock> childBlocks = masterPlan.getChilds(parentBlockId);
- for (ExecutionBlock childBlock : childBlocks) {
- Stage childStage = context.getStage(childBlock.getId());
- tableStatses.add(childStage.getResultStats());
- }
- return StatisticsUtil.aggregateTableStat(tableStatses);
- }
-
- public static void scheduleRangeShuffledFetches(TaskSchedulerContext schedulerContext, MasterPlan masterPlan,
- Stage stage, DataChannel channel, int maxNum)
- throws IOException {
- ExecutionBlock execBlock = stage.getBlock();
- ScanNode scan = execBlock.getScanNodes()[0];
- Path tablePath;
- tablePath = ((FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf()))
- .getTablePath(scan.getTableName());
-
- ExecutionBlock sampleChildBlock = masterPlan.getChild(stage.getId(), 0);
- SortNode sortNode = PlannerUtil.findTopNode(sampleChildBlock.getPlan(), NodeType.SORT);
- SortSpec [] sortSpecs = sortNode.getSortKeys();
- Schema sortSchema = new Schema(channel.getShuffleKeys());
-
- TupleRange[] ranges;
- int determinedTaskNum;
-
- // calculate the number of maximum query ranges
- TableStats totalStat = computeChildBlocksStats(stage.getContext(), masterPlan, stage.getId());
-
- // If there is an empty table in inner join, it should return zero rows.
- if (totalStat.getNumBytes() == 0 && totalStat.getColumnStats().size() == 0) {
- return;
- }
- TupleRange mergedRange = TupleUtil.columnStatToRange(sortSpecs, sortSchema, totalStat.getColumnStats(), false);
-
- if (sortNode.getSortPurpose() == SortPurpose.STORAGE_SPECIFIED) {
- StoreType storeType = PlannerUtil.getStoreType(masterPlan.getLogicalPlan());
- CatalogService catalog = stage.getContext().getQueryMasterContext().getWorkerContext().getCatalog();
- LogicalRootNode rootNode = masterPlan.getLogicalPlan().getRootBlock().getRoot();
- TableDesc tableDesc = PlannerUtil.getTableDesc(catalog, rootNode.getChild());
- if (tableDesc == null) {
- throw new IOException("Can't get table meta data from catalog: " +
- PlannerUtil.getStoreTableName(masterPlan.getLogicalPlan()));
- }
- ranges = StorageManager.getStorageManager(stage.getContext().getConf(), storeType)
- .getInsertSortRanges(stage.getContext().getQueryContext(), tableDesc,
- sortNode.getInSchema(), sortSpecs,
- mergedRange);
- determinedTaskNum = ranges.length;
- } else {
- RangePartitionAlgorithm partitioner = new UniformRangePartition(mergedRange, sortSpecs);
- BigInteger card = partitioner.getTotalCardinality();
-
- // if the number of the range cardinality is less than the desired number of tasks,
- // we set the the number of tasks to the number of range cardinality.
- if (card.compareTo(BigInteger.valueOf(maxNum)) < 0) {
- LOG.info(stage.getId() + ", The range cardinality (" + card
- + ") is less then the desired number of tasks (" + maxNum + ")");
- determinedTaskNum = card.intValue();
- } else {
- determinedTaskNum = maxNum;
- }
-
- LOG.info(stage.getId() + ", Try to divide " + mergedRange + " into " + determinedTaskNum +
- " sub ranges (total units: " + determinedTaskNum + ")");
- ranges = partitioner.partition(determinedTaskNum);
- if (ranges == null || ranges.length == 0) {
- LOG.warn(stage.getId() + " no range infos.");
- }
- TupleUtil.setMaxRangeIfNull(sortSpecs, sortSchema, totalStat.getColumnStats(), ranges);
- if (LOG.isDebugEnabled()) {
- if (ranges != null) {
- for (TupleRange eachRange : ranges) {
- LOG.debug(stage.getId() + " range: " + eachRange.getStart() + " ~ " + eachRange.getEnd());
- }
- }
- }
- }
-
- FileFragment dummyFragment = new FileFragment(scan.getTableName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
- Stage.scheduleFragment(stage, dummyFragment);
-
- List<FetchImpl> fetches = new ArrayList<FetchImpl>();
- List<ExecutionBlock> childBlocks = masterPlan.getChilds(stage.getId());
- for (ExecutionBlock childBlock : childBlocks) {
- Stage childExecSM = stage.getContext().getStage(childBlock.getId());
- for (Task qu : childExecSM.getTasks()) {
- for (IntermediateEntry p : qu.getIntermediateData()) {
- FetchImpl fetch = new FetchImpl(p.getPullHost(), RANGE_SHUFFLE, childBlock.getId(), 0);
- fetch.addPart(p.getTaskId(), p.getAttemptId());
- fetches.add(fetch);
- }
- }
- }
-
- boolean ascendingFirstKey = sortSpecs[0].isAscending();
- SortedMap<TupleRange, Collection<FetchImpl>> map;
- if (ascendingFirstKey) {
- map = new TreeMap<TupleRange, Collection<FetchImpl>>();
- } else {
- map = new TreeMap<TupleRange, Collection<FetchImpl>>(new TupleRange.DescendingTupleRangeComparator());
- }
-
- Set<FetchImpl> fetchSet;
- try {
- RowStoreUtil.RowStoreEncoder encoder = RowStoreUtil.createEncoder(sortSchema);
- for (int i = 0; i < ranges.length; i++) {
- fetchSet = new HashSet<FetchImpl>();
- for (FetchImpl fetch: fetches) {
- String rangeParam =
- TupleUtil.rangeToQuery(ranges[i], ascendingFirstKey ? i == (ranges.length - 1) : i == 0, encoder);
- FetchImpl copy = null;
- try {
- copy = fetch.clone();
- } catch (CloneNotSupportedException e) {
- throw new RuntimeException(e);
- }
- copy.setRangeParams(rangeParam);
- fetchSet.add(copy);
- }
- map.put(ranges[i], fetchSet);
- }
-
- } catch (UnsupportedEncodingException e) {
- LOG.error(e);
- }
-
- scheduleFetchesByRoundRobin(stage, map, scan.getTableName(), determinedTaskNum);
-
- schedulerContext.setEstimatedTaskNum(determinedTaskNum);
- }
-
- public static void scheduleFetchesByRoundRobin(Stage stage, Map<?, Collection<FetchImpl>> partitions,
- String tableName, int num) {
- int i;
- Map<String, List<FetchImpl>>[] fetchesArray = new Map[num];
- for (i = 0; i < num; i++) {
- fetchesArray[i] = new HashMap<String, List<FetchImpl>>();
- }
- i = 0;
- for (Entry<?, Collection<FetchImpl>> entry : partitions.entrySet()) {
- Collection<FetchImpl> value = entry.getValue();
- TUtil.putCollectionToNestedList(fetchesArray[i++], tableName, value);
- if (i == num) i = 0;
- }
- for (Map<String, List<FetchImpl>> eachFetches : fetchesArray) {
- Stage.scheduleFetches(stage, eachFetches);
- }
- }
-
- @VisibleForTesting
- public static class FetchGroupMeta {
- long totalVolume;
- List<FetchImpl> fetchUrls;
-
- public FetchGroupMeta(long volume, FetchImpl fetchUrls) {
- this.totalVolume = volume;
- this.fetchUrls = Lists.newArrayList(fetchUrls);
- }
-
- public FetchGroupMeta addFetche(FetchImpl fetches) {
- this.fetchUrls.add(fetches);
- return this;
- }
-
- public void increaseVolume(long volume) {
- this.totalVolume += volume;
- }
-
- public long getVolume() {
- return totalVolume;
- }
-
- }
-
- public static void scheduleHashShuffledFetches(TaskSchedulerContext schedulerContext, MasterPlan masterPlan,
- Stage stage, DataChannel channel,
- int maxNum) throws IOException {
- ExecutionBlock execBlock = stage.getBlock();
- ScanNode scan = execBlock.getScanNodes()[0];
- Path tablePath;
- tablePath = ((FileStorageManager)StorageManager.getFileStorageManager(stage.getContext().getConf()))
- .getTablePath(scan.getTableName());
-
- Fragment frag = new FileFragment(scan.getCanonicalName(), tablePath, 0, 0, new String[]{UNKNOWN_HOST});
- List<Fragment> fragments = new ArrayList<Fragment>();
- fragments.add(frag);
- Stage.scheduleFragments(stage, fragments);
-
- Map<Integer, FetchGroupMeta> finalFetches = new HashMap<Integer, FetchGroupMeta>();
- Map<ExecutionBlockId, List<IntermediateEntry>> intermediates = new HashMap<ExecutionBlockId,
- List<IntermediateEntry>>();
-
- for (ExecutionBlock block : masterPlan.getChilds(execBlock)) {
- List<IntermediateEntry> partitions = new ArrayList<IntermediateEntry>();
- partitions.addAll(stage.getContext().getStage(block.getId()).getHashShuffleIntermediateEntries());
-
- // In scattered hash shuffle, Collecting each IntermediateEntry
- if (channel.getShuffleType() == SCATTERED_HASH_SHUFFLE) {
- if (intermediates.containsKey(block.getId())) {
- intermediates.get(block.getId()).addAll(partitions);
- } else {
- intermediates.put(block.getId(), partitions);
- }
- }
-
- // make FetchImpl per PullServer, PartId
- Map<Integer, List<IntermediateEntry>> hashed = hashByKey(partitions);
- for (Entry<Integer, List<IntermediateEntry>> interm : hashed.entrySet()) {
- Map<Task.PullHost, List<IntermediateEntry>> hashedByHost = hashByHost(interm.getValue());
- for (Entry<Task.PullHost, List<IntermediateEntry>> e : hashedByHost.entrySet()) {
-
- FetchImpl fetch = new FetchImpl(e.getKey(), channel.getShuffleType(),
- block.getId(), interm.getKey(), e.getValue());
-
- long volumeSum = 0;
- for (IntermediateEntry ie : e.getValue()) {
- volumeSum += ie.getVolume();
- }
-
- if (finalFetches.containsKey(interm.getKey())) {
- finalFetches.get(interm.getKey()).addFetche(fetch).increaseVolume(volumeSum);
- } else {
- finalFetches.put(interm.getKey(), new FetchGroupMeta(volumeSum, fetch));
- }
- }
- }
- }
-
- int groupingColumns = 0;
- LogicalNode[] groupbyNodes = PlannerUtil.findAllNodes(stage.getBlock().getPlan(),
- new NodeType[]{NodeType.GROUP_BY, NodeType.DISTINCT_GROUP_BY});
- if (groupbyNodes != null && groupbyNodes.length > 0) {
- LogicalNode bottomNode = groupbyNodes[0];
- if (bottomNode.getType() == NodeType.GROUP_BY) {
- groupingColumns = ((GroupbyNode)bottomNode).getGroupingColumns().length;
- } else if (bottomNode.getType() == NodeType.DISTINCT_GROUP_BY) {
- DistinctGroupbyNode distinctNode = PlannerUtil.findMostBottomNode(stage.getBlock().getPlan(), NodeType.DISTINCT_GROUP_BY);
- if (distinctNode == null) {
- LOG.warn(stage.getId() + ", Can't find current DistinctGroupbyNode");
- distinctNode = (DistinctGroupbyNode)bottomNode;
- }
- groupingColumns = distinctNode.getGroupingColumns().length;
-
- Enforcer enforcer = execBlock.getEnforcer();
- EnforceProperty property = PhysicalPlannerImpl.getAlgorithmEnforceProperty(enforcer, distinctNode);
- if (property != null) {
- if (property.getDistinct().getIsMultipleAggregation()) {
- MultipleAggregationStage mulAggStage = property.getDistinct().getMultipleAggregationStage();
- if (mulAggStage != MultipleAggregationStage.THRID_STAGE) {
- groupingColumns = distinctNode.getOutSchema().size();
- }
- }
- }
- }
- }
- // get a proper number of tasks
- int determinedTaskNum = Math.min(maxNum, finalFetches.size());
- LOG.info(stage.getId() + ", ScheduleHashShuffledFetches - Max num=" + maxNum + ", finalFetchURI=" + finalFetches.size());
-
- if (groupingColumns == 0) {
- determinedTaskNum = 1;
- LOG.info(stage.getId() + ", No Grouping Column - determinedTaskNum is set to 1");
- } else {
- TableStats totalStat = computeChildBlocksStats(stage.getContext(), masterPlan, stage.getId());
- if (totalStat.getNumRows() == 0) {
- determinedTaskNum = 1;
- }
- }
-
- // set the proper number of tasks to the estimated task num
- if (channel.getShuffleType() == SCATTERED_HASH_SHUFFLE) {
- scheduleScatteredHashShuffleFetches(schedulerContext, stage, intermediates,
- scan.getTableName());
- } else {
- schedulerContext.setEstimatedTaskNum(determinedTaskNum);
- // divide fetch uris into the the proper number of tasks according to volumes
- scheduleFetchesByEvenDistributedVolumes(stage, finalFetches, scan.getTableName(), determinedTaskNum);
- LOG.info(stage.getId() + ", DeterminedTaskNum : " + determinedTaskNum);
- }
- }
-
- public static Pair<Long [], Map<String, List<FetchImpl>>[]> makeEvenDistributedFetchImpl(
- Map<Integer, FetchGroupMeta> partitions, String tableName, int num) {
-
- // Sort fetchGroupMeta in a descending order of data volumes.
- List<FetchGroupMeta> fetchGroupMetaList = Lists.newArrayList(partitions.values());
- Collections.sort(fetchGroupMetaList, new Comparator<FetchGroupMeta>() {
- @Override
- public int compare(FetchGroupMeta o1, FetchGroupMeta o2) {
- return o1.getVolume() < o2.getVolume() ? 1 : (o1.getVolume() > o2.getVolume() ? -1 : 0);
- }
- });
-
- // Initialize containers
- Map<String, List<FetchImpl>>[] fetchesArray = new Map[num];
- Long [] assignedVolumes = new Long[num];
- // initialization
- for (int i = 0; i < num; i++) {
- fetchesArray[i] = new HashMap<String, List<FetchImpl>>();
- assignedVolumes[i] = 0l;
- }
-
- // This algorithm assignes bigger first manner by using a sorted iterator. It is a kind of greedy manner.
- // Its complexity is O(n). Since FetchGroup can be more than tens of thousands, we should consider its complexity.
- // In terms of this point, it will show reasonable performance and results. even though it is not an optimal
- // algorithm.
- Iterator<FetchGroupMeta> iterator = fetchGroupMetaList.iterator();
-
- int p = 0;
- while(iterator.hasNext()) {
- while (p < num && iterator.hasNext()) {
- FetchGroupMeta fetchGroupMeta = iterator.next();
- assignedVolumes[p] += fetchGroupMeta.getVolume();
-
- TUtil.putCollectionToNestedList(fetchesArray[p], tableName, fetchGroupMeta.fetchUrls);
- p++;
- }
-
- p = num - 1;
- while (p > 0 && iterator.hasNext()) {
- FetchGroupMeta fetchGroupMeta = iterator.next();
- assignedVolumes[p] += fetchGroupMeta.getVolume();
- TUtil.putCollectionToNestedList(fetchesArray[p], tableName, fetchGroupMeta.fetchUrls);
-
- // While the current one is smaller than next one, it adds additional fetches to current one.
- while(iterator.hasNext() && assignedVolumes[p - 1] > assignedVolumes[p]) {
- FetchGroupMeta additionalFetchGroup = iterator.next();
- assignedVolumes[p] += additionalFetchGroup.getVolume();
- TUtil.putCollectionToNestedList(fetchesArray[p], tableName, additionalFetchGroup.fetchUrls);
- }
-
- p--;
- }
- }
-
- return new Pair<Long[], Map<String, List<FetchImpl>>[]>(assignedVolumes, fetchesArray);
- }
-
- public static void scheduleFetchesByEvenDistributedVolumes(Stage stage, Map<Integer, FetchGroupMeta> partitions,
- String tableName, int num) {
- Map<String, List<FetchImpl>>[] fetchsArray = makeEvenDistributedFetchImpl(partitions, tableName, num).getSecond();
- // Schedule FetchImpls
- for (Map<String, List<FetchImpl>> eachFetches : fetchsArray) {
- Stage.scheduleFetches(stage, eachFetches);
- }
- }
-
- // Scattered hash shuffle hashes the key columns and groups the hash keys associated with
- // the same hash key. Then, if the volume of a group is larger
- // than $DIST_QUERY_TABLE_PARTITION_VOLUME, it divides the group into more than two sub groups
- // according to $DIST_QUERY_TABLE_PARTITION_VOLUME (default size = 256MB).
- // As a result, each group size always becomes the less than or equal
- // to $DIST_QUERY_TABLE_PARTITION_VOLUME. Finally, each subgroup is assigned to a query unit.
- // It is usually used for writing partitioned tables.
- public static void scheduleScatteredHashShuffleFetches(TaskSchedulerContext schedulerContext,
- Stage stage, Map<ExecutionBlockId, List<IntermediateEntry>> intermediates,
- String tableName) {
- long splitVolume = StorageUnit.MB *
- stage.getMasterPlan().getContext().getLong(SessionVars.TABLE_PARTITION_PER_SHUFFLE_SIZE);
- long pageSize = StorageUnit.MB *
- stage.getContext().getConf().getIntVar(ConfVars.SHUFFLE_HASH_APPENDER_PAGE_VOLUME); // in bytes
- if (pageSize >= splitVolume) {
- throw new RuntimeException("tajo.dist-query.table-partition.task-volume-mb should be great than " +
- "tajo.shuffle.hash.appender.page.volumn-mb");
- }
- List<List<FetchImpl>> fetches = new ArrayList<List<FetchImpl>>();
-
- long totalIntermediateSize = 0L;
- for (Entry<ExecutionBlockId, List<IntermediateEntry>> listEntry : intermediates.entrySet()) {
- // merge by PartitionId
- Map<Integer, List<IntermediateEntry>> partitionIntermMap = new HashMap<Integer, List<IntermediateEntry>>();
- for (IntermediateEntry eachInterm: listEntry.getValue()) {
- totalIntermediateSize += eachInterm.getVolume();
- int partId = eachInterm.getPartId();
- List<IntermediateEntry> partitionInterms = partitionIntermMap.get(partId);
- if (partitionInterms == null) {
- partitionInterms = TUtil.newList(eachInterm);
- partitionIntermMap.put(partId, partitionInterms);
- } else {
- partitionInterms.add(eachInterm);
- }
- }
-
- // Grouping or splitting to fit $DIST_QUERY_TABLE_PARTITION_VOLUME size
- for (List<IntermediateEntry> partitionEntries : partitionIntermMap.values()) {
- List<List<FetchImpl>> eachFetches = splitOrMergeIntermediates(listEntry.getKey(), partitionEntries,
- splitVolume, pageSize);
- if (eachFetches != null && !eachFetches.isEmpty()) {
- fetches.addAll(eachFetches);
- }
- }
- }
-
- schedulerContext.setEstimatedTaskNum(fetches.size());
-
- int i = 0;
- Map<String, List<FetchImpl>>[] fetchesArray = new Map[fetches.size()];
- for(List<FetchImpl> entry : fetches) {
- fetchesArray[i] = new HashMap<String, List<FetchImpl>>();
- fetchesArray[i].put(tableName, entry);
-
- Stage.scheduleFetches(stage, fetchesArray[i]);
- i++;
- }
-
- LOG.info(stage.getId()
- + ", ShuffleType:" + SCATTERED_HASH_SHUFFLE.name()
- + ", Intermediate Size: " + totalIntermediateSize
- + ", splitSize: " + splitVolume
- + ", DeterminedTaskNum: " + fetches.size());
- }
-
- /**
- * If a IntermediateEntry is large than splitVolume, List<FetchImpl> has single element.
- * @param ebId
- * @param entries
- * @param splitVolume
- * @return
- */
- public static List<List<FetchImpl>> splitOrMergeIntermediates(
- ExecutionBlockId ebId, List<IntermediateEntry> entries, long splitVolume, long pageSize) {
- // Each List<FetchImpl> has splitVolume size.
- List<List<FetchImpl>> fetches = new ArrayList<List<FetchImpl>>();
-
- Iterator<IntermediateEntry> iter = entries.iterator();
- if (!iter.hasNext()) {
- return null;
- }
- List<FetchImpl> fetchListForSingleTask = new ArrayList<FetchImpl>();
- long fetchListVolume = 0;
-
- while (iter.hasNext()) {
- IntermediateEntry currentInterm = iter.next();
-
- long firstSplitVolume = splitVolume - fetchListVolume;
- if (firstSplitVolume < pageSize) {
- firstSplitVolume = splitVolume;
- }
-
- //Each Pair object in the splits variable is assigned to the next ExectionBlock's task.
- //The first long value is a offset of the intermediate file and the second long value is length.
- List<Pair<Long, Long>> splits = currentInterm.split(firstSplitVolume, splitVolume);
- if (splits == null || splits.isEmpty()) {
- break;
- }
-
- for (Pair<Long, Long> eachSplit: splits) {
- if (fetchListVolume > 0 && fetchListVolume + eachSplit.getSecond() >= splitVolume) {
- if (!fetchListForSingleTask.isEmpty()) {
- fetches.add(fetchListForSingleTask);
- }
- fetchListForSingleTask = new ArrayList<FetchImpl>();
- fetchListVolume = 0;
- }
- FetchImpl fetch = new FetchImpl(currentInterm.getPullHost(), SCATTERED_HASH_SHUFFLE,
- ebId, currentInterm.getPartId(), TUtil.newList(currentInterm));
- fetch.setOffset(eachSplit.getFirst());
- fetch.setLength(eachSplit.getSecond());
- fetchListForSingleTask.add(fetch);
- fetchListVolume += eachSplit.getSecond();
- }
- }
- if (!fetchListForSingleTask.isEmpty()) {
- fetches.add(fetchListForSingleTask);
- }
- return fetches;
- }
-
- public static List<URI> createFetchURL(FetchImpl fetch, boolean includeParts) {
- String scheme = "http://";
-
- StringBuilder urlPrefix = new StringBuilder(scheme);
- urlPrefix.append(fetch.getPullHost().getHost()).append(":").append(fetch.getPullHost().getPort()).append("/?")
- .append("qid=").append(fetch.getExecutionBlockId().getQueryId().toString())
- .append("&sid=").append(fetch.getExecutionBlockId().getId())
- .append("&p=").append(fetch.getPartitionId())
- .append("&type=");
- if (fetch.getType() == HASH_SHUFFLE) {
- urlPrefix.append("h");
- } else if (fetch.getType() == RANGE_SHUFFLE) {
- urlPrefix.append("r").append("&").append(fetch.getRangeParams());
- } else if (fetch.getType() == SCATTERED_HASH_SHUFFLE) {
- urlPrefix.append("s");
- }
-
- if (fetch.getLength() >= 0) {
- urlPrefix.append("&offset=").append(fetch.getOffset()).append("&length=").append(fetch.getLength());
- }
-
- List<URI> fetchURLs = new ArrayList<URI>();
- if(includeParts) {
- if (fetch.getType() == HASH_SHUFFLE || fetch.getType() == SCATTERED_HASH_SHUFFLE) {
- fetchURLs.add(URI.create(urlPrefix.toString()));
- } else {
- // If the get request is longer than 2000 characters,
- // the long request uri may cause HTTP Status Code - 414 Request-URI Too Long.
- // Refer to http://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html#sec10.4.15
- // The below code transforms a long request to multiple requests.
- List<String> taskIdsParams = new ArrayList<String>();
- StringBuilder taskIdListBuilder = new StringBuilder();
- List<Integer> taskIds = fetch.getTaskIds();
- List<Integer> attemptIds = fetch.getAttemptIds();
- boolean first = true;
-
- for (int i = 0; i < taskIds.size(); i++) {
- StringBuilder taskAttemptId = new StringBuilder();
-
- if (!first) { // when comma is added?
- taskAttemptId.append(",");
- } else {
- first = false;
- }
-
- int taskId = taskIds.get(i);
- if (taskId < 0) {
- // In the case of hash shuffle each partition has single shuffle file per worker.
- // TODO If file is large, consider multiple fetching(shuffle file can be split)
- continue;
- }
- int attemptId = attemptIds.get(i);
- taskAttemptId.append(taskId).append("_").append(attemptId);
-
- if (taskIdListBuilder.length() + taskAttemptId.length()
- > HTTP_REQUEST_MAXIMUM_LENGTH) {
- taskIdsParams.add(taskIdListBuilder.toString());
- taskIdListBuilder = new StringBuilder(taskId + "_" + attemptId);
- } else {
- taskIdListBuilder.append(taskAttemptId);
- }
- }
- // if the url params remain
- if (taskIdListBuilder.length() > 0) {
- taskIdsParams.add(taskIdListBuilder.toString());
- }
- urlPrefix.append("&ta=");
- for (String param : taskIdsParams) {
- fetchURLs.add(URI.create(urlPrefix + param));
- }
- }
- } else {
- fetchURLs.add(URI.create(urlPrefix.toString()));
- }
-
- return fetchURLs;
- }
-
- public static Map<Integer, List<IntermediateEntry>> hashByKey(List<IntermediateEntry> entries) {
- Map<Integer, List<IntermediateEntry>> hashed = new HashMap<Integer, List<IntermediateEntry>>();
- for (IntermediateEntry entry : entries) {
- if (hashed.containsKey(entry.getPartId())) {
- hashed.get(entry.getPartId()).add(entry);
- } else {
- hashed.put(entry.getPartId(), TUtil.newList(entry));
- }
- }
-
- return hashed;
- }
-
- public static Map<Task.PullHost, List<IntermediateEntry>> hashByHost(List<IntermediateEntry> entries) {
- Map<Task.PullHost, List<IntermediateEntry>> hashed = new HashMap<Task.PullHost, List<IntermediateEntry>>();
-
- Task.PullHost host;
- for (IntermediateEntry entry : entries) {
- host = entry.getPullHost();
- if (hashed.containsKey(host)) {
- hashed.get(host).add(entry);
- } else {
- hashed.put(host, TUtil.newList(entry));
- }
- }
-
- return hashed;
- }
-
- public static Stage setShuffleOutputNumForTwoPhase(Stage stage, final int desiredNum, DataChannel channel) {
- ExecutionBlock execBlock = stage.getBlock();
- Column[] keys;
- // if the next query is join,
- // set the partition number for the current logicalUnit
- // TODO: the union handling is required when a join has unions as its child
- MasterPlan masterPlan = stage.getMasterPlan();
- keys = channel.getShuffleKeys();
- if (!masterPlan.isRoot(stage.getBlock()) ) {
- ExecutionBlock parentBlock = masterPlan.getParent(stage.getBlock());
- if (parentBlock.getPlan().getType() == NodeType.JOIN) {
- channel.setShuffleOutputNum(desiredNum);
- }
- }
-
- // set the partition number for group by and sort
- if (channel.getShuffleType() == HASH_SHUFFLE) {
- if (execBlock.getPlan().getType() == NodeType.GROUP_BY ||
- execBlock.getPlan().getType() == NodeType.DISTINCT_GROUP_BY) {
- keys = channel.getShuffleKeys();
- }
- } else if (channel.getShuffleType() == RANGE_SHUFFLE) {
- if (execBlock.getPlan().getType() == NodeType.SORT) {
- SortNode sort = (SortNode) execBlock.getPlan();
- keys = new Column[sort.getSortKeys().length];
- for (int i = 0; i < keys.length; i++) {
- keys[i] = sort.getSortKeys()[i].getSortKey();
- }
- }
- }
- if (keys != null) {
- if (keys.length == 0) {
- channel.setShuffleKeys(new Column[]{});
- channel.setShuffleOutputNum(1);
- } else {
- channel.setShuffleKeys(keys);
- channel.setShuffleOutputNum(desiredNum);
- }
- }
- return stage;
- }
-}
[03/16] tajo git commit: TAJO-1288: Refactoring
org.apache.tajo.master package.
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java
new file mode 100644
index 0000000..86c49b4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskAttempt.java
@@ -0,0 +1,443 @@
+/**
+ * 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.tajo.querymaster;
+
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.state.*;
+import org.apache.tajo.TaskAttemptId;
+import org.apache.tajo.TajoProtos.TaskAttemptState;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStats;
+import org.apache.tajo.ipc.TajoWorkerProtocol.TaskCompletionReport;
+import org.apache.tajo.master.cluster.WorkerConnectionInfo;
+import org.apache.tajo.master.event.*;
+import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext;
+import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
+import org.apache.tajo.querymaster.Task.IntermediateEntry;
+import org.apache.tajo.querymaster.Task.PullHost;
+import org.apache.tajo.master.container.TajoContainerId;
+
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static org.apache.tajo.ipc.TajoWorkerProtocol.ShuffleFileOutput;
+
+public class TaskAttempt implements EventHandler<TaskAttemptEvent> {
+
+ private static final Log LOG = LogFactory.getLog(TaskAttempt.class);
+
+ private final static int EXPIRE_TIME = 15000;
+
+ private final TaskAttemptId id;
+ private final Task task;
+ final EventHandler eventHandler;
+
+ private TajoContainerId containerId;
+ private WorkerConnectionInfo workerConnectionInfo;
+ private int expire;
+
+ private final Lock readLock;
+ private final Lock writeLock;
+
+ private final List<String> diagnostics = new ArrayList<String>();
+
+ private final TaskAttemptScheduleContext scheduleContext;
+
+ private float progress;
+ private CatalogProtos.TableStatsProto inputStats;
+ private CatalogProtos.TableStatsProto resultStats;
+
+ protected static final StateMachineFactory
+ <TaskAttempt, TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
+ stateMachineFactory = new StateMachineFactory
+ <TaskAttempt, TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
+ (TaskAttemptState.TA_NEW)
+
+ // Transitions from TA_NEW state
+ .addTransition(TaskAttemptState.TA_NEW, TaskAttemptState.TA_UNASSIGNED,
+ TaskAttemptEventType.TA_SCHEDULE, new TaskAttemptScheduleTransition())
+ .addTransition(TaskAttemptState.TA_NEW, TaskAttemptState.TA_UNASSIGNED,
+ TaskAttemptEventType.TA_RESCHEDULE, new TaskAttemptScheduleTransition())
+ .addTransition(TaskAttemptState.TA_NEW, TaskAttemptState.TA_KILLED,
+ TaskAttemptEventType.TA_KILL,
+ new TaskKilledCompleteTransition())
+
+ // Transitions from TA_UNASSIGNED state
+ .addTransition(TaskAttemptState.TA_UNASSIGNED, TaskAttemptState.TA_ASSIGNED,
+ TaskAttemptEventType.TA_ASSIGNED,
+ new LaunchTransition())
+ .addTransition(TaskAttemptState.TA_UNASSIGNED, TaskAttemptState.TA_KILL_WAIT,
+ TaskAttemptEventType.TA_KILL,
+ new KillUnassignedTaskTransition())
+
+ // Transitions from TA_ASSIGNED state
+ .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_ASSIGNED,
+ TaskAttemptEventType.TA_ASSIGNED, new AlreadyAssignedTransition())
+ .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_KILL_WAIT,
+ TaskAttemptEventType.TA_KILL,
+ new KillTaskTransition())
+ .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_KILLED,
+ TaskAttemptEventType.TA_KILL,
+ new KillTaskTransition())
+ .addTransition(TaskAttemptState.TA_ASSIGNED,
+ EnumSet.of(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_KILLED),
+ TaskAttemptEventType.TA_UPDATE, new StatusUpdateTransition())
+ .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_SUCCEEDED,
+ TaskAttemptEventType.TA_DONE, new SucceededTransition())
+ .addTransition(TaskAttemptState.TA_ASSIGNED, TaskAttemptState.TA_FAILED,
+ TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
+
+ // Transitions from TA_RUNNING state
+ .addTransition(TaskAttemptState.TA_RUNNING,
+ EnumSet.of(TaskAttemptState.TA_RUNNING),
+ TaskAttemptEventType.TA_UPDATE, new StatusUpdateTransition())
+ .addTransition(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_KILL_WAIT,
+ TaskAttemptEventType.TA_KILL,
+ new KillTaskTransition())
+ .addTransition(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_SUCCEEDED,
+ TaskAttemptEventType.TA_DONE, new SucceededTransition())
+ .addTransition(TaskAttemptState.TA_RUNNING, TaskAttemptState.TA_FAILED,
+ TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
+
+ .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILLED,
+ TaskAttemptEventType.TA_LOCAL_KILLED,
+ new TaskKilledCompleteTransition())
+ .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILL_WAIT,
+ TaskAttemptEventType.TA_ASSIGNED,
+ new KillTaskTransition())
+ .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILLED,
+ TaskAttemptEventType.TA_SCHEDULE_CANCELED,
+ new TaskKilledCompleteTransition())
+ .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILLED,
+ TaskAttemptEventType.TA_DONE,
+ new TaskKilledCompleteTransition())
+ .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_FAILED,
+ TaskAttemptEventType.TA_FATAL_ERROR)
+ .addTransition(TaskAttemptState.TA_KILL_WAIT, TaskAttemptState.TA_KILL_WAIT,
+ EnumSet.of(
+ TaskAttemptEventType.TA_KILL,
+ TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE,
+ TaskAttemptEventType.TA_UPDATE))
+
+ // Transitions from TA_SUCCEEDED state
+ .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_SUCCEEDED,
+ TaskAttemptEventType.TA_UPDATE)
+ .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_SUCCEEDED,
+ TaskAttemptEventType.TA_DONE, new AlreadyDoneTransition())
+ .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_FAILED,
+ TaskAttemptEventType.TA_FATAL_ERROR, new FailedTransition())
+ // Ignore-able transitions
+ .addTransition(TaskAttemptState.TA_SUCCEEDED, TaskAttemptState.TA_SUCCEEDED,
+ TaskAttemptEventType.TA_KILL)
+
+ // Transitions from TA_KILLED state
+ .addTransition(TaskAttemptState.TA_KILLED, TaskAttemptState.TA_KILLED,
+ TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE)
+ // Ignore-able transitions
+ .addTransition(TaskAttemptState.TA_KILLED, TaskAttemptState.TA_KILLED,
+ EnumSet.of(
+ TaskAttemptEventType.TA_UPDATE))
+ .addTransition(TaskAttemptState.TA_KILLED, TaskAttemptState.TA_KILLED,
+ EnumSet.of(
+ TaskAttemptEventType.TA_LOCAL_KILLED,
+ TaskAttemptEventType.TA_KILL,
+ TaskAttemptEventType.TA_ASSIGNED,
+ TaskAttemptEventType.TA_DONE),
+ new TaskKilledCompleteTransition())
+ .installTopology();
+
+ private final StateMachine<TaskAttemptState, TaskAttemptEventType, TaskAttemptEvent>
+ stateMachine;
+
+
+ public TaskAttempt(final TaskAttemptScheduleContext scheduleContext,
+ final TaskAttemptId id, final Task task,
+ final EventHandler eventHandler) {
+ this.scheduleContext = scheduleContext;
+ this.id = id;
+ this.expire = TaskAttempt.EXPIRE_TIME;
+ this.task = task;
+ this.eventHandler = eventHandler;
+
+ ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+ this.readLock = readWriteLock.readLock();
+ this.writeLock = readWriteLock.writeLock();
+
+ stateMachine = stateMachineFactory.make(this);
+ }
+
+ public TaskAttemptState getState() {
+ readLock.lock();
+ try {
+ return stateMachine.getCurrentState();
+ } finally {
+ readLock.unlock();
+ }
+ }
+
+ public TaskAttemptId getId() {
+ return this.id;
+ }
+
+ public boolean isLeafTask() {
+ return this.task.isLeafTask();
+ }
+
+ public Task getTask() {
+ return this.task;
+ }
+
+ public WorkerConnectionInfo getWorkerConnectionInfo() {
+ return this.workerConnectionInfo;
+ }
+
+ public void setContainerId(TajoContainerId containerId) {
+ this.containerId = containerId;
+ }
+
+ public synchronized void setExpireTime(int expire) {
+ this.expire = expire;
+ }
+
+ public synchronized void updateExpireTime(int period) {
+ this.setExpireTime(this.expire - period);
+ }
+
+ public synchronized void resetExpireTime() {
+ this.setExpireTime(TaskAttempt.EXPIRE_TIME);
+ }
+
+ public int getLeftTime() {
+ return this.expire;
+ }
+
+ public float getProgress() {
+ return progress;
+ }
+
+ public TableStats getInputStats() {
+ if (inputStats == null) {
+ return null;
+ }
+
+ return new TableStats(inputStats);
+ }
+
+ public TableStats getResultStats() {
+ if (resultStats == null) {
+ return null;
+ }
+ return new TableStats(resultStats);
+ }
+
+ private void fillTaskStatistics(TaskCompletionReport report) {
+ this.progress = 1.0f;
+
+ List<IntermediateEntry> partitions = new ArrayList<IntermediateEntry>();
+
+ if (report.getShuffleFileOutputsCount() > 0) {
+ this.getTask().setShuffleFileOutputs(report.getShuffleFileOutputsList());
+
+ PullHost host = new PullHost(getWorkerConnectionInfo().getHost(), getWorkerConnectionInfo().getPullServerPort());
+ for (ShuffleFileOutput p : report.getShuffleFileOutputsList()) {
+ IntermediateEntry entry = new IntermediateEntry(getId().getTaskId().getId(),
+ getId().getId(), p.getPartId(), host, p.getVolume());
+ partitions.add(entry);
+ }
+ }
+ this.getTask().setIntermediateData(partitions);
+
+ if (report.hasInputStats()) {
+ this.inputStats = report.getInputStats();
+ }
+ if (report.hasResultStats()) {
+ this.resultStats = report.getResultStats();
+ this.getTask().setStats(new TableStats(resultStats));
+ }
+ }
+
+ private static class TaskAttemptScheduleTransition implements
+ SingleArcTransition<TaskAttempt, TaskAttemptEvent> {
+
+ @Override
+ public void transition(TaskAttempt taskAttempt, TaskAttemptEvent taskAttemptEvent) {
+ taskAttempt.eventHandler.handle(new TaskAttemptToSchedulerEvent(
+ EventType.T_SCHEDULE, taskAttempt.getTask().getId().getExecutionBlockId(),
+ taskAttempt.scheduleContext, taskAttempt));
+ }
+ }
+
+ private static class KillUnassignedTaskTransition implements
+ SingleArcTransition<TaskAttempt, TaskAttemptEvent> {
+
+ @Override
+ public void transition(TaskAttempt taskAttempt, TaskAttemptEvent taskAttemptEvent) {
+ taskAttempt.eventHandler.handle(new TaskAttemptToSchedulerEvent(
+ EventType.T_SCHEDULE_CANCEL, taskAttempt.getTask().getId().getExecutionBlockId(),
+ taskAttempt.scheduleContext, taskAttempt));
+ }
+ }
+
+ private static class LaunchTransition
+ implements SingleArcTransition<TaskAttempt, TaskAttemptEvent> {
+
+ @Override
+ public void transition(TaskAttempt taskAttempt,
+ TaskAttemptEvent event) {
+ TaskAttemptAssignedEvent castEvent = (TaskAttemptAssignedEvent) event;
+ taskAttempt.containerId = castEvent.getContainerId();
+ taskAttempt.workerConnectionInfo = castEvent.getWorkerConnectionInfo();
+ taskAttempt.eventHandler.handle(
+ new TaskTAttemptEvent(taskAttempt.getId(),
+ TaskEventType.T_ATTEMPT_LAUNCHED));
+ }
+ }
+
+ private static class TaskKilledCompleteTransition implements SingleArcTransition<TaskAttempt, TaskAttemptEvent> {
+
+ @Override
+ public void transition(TaskAttempt taskAttempt,
+ TaskAttemptEvent event) {
+ taskAttempt.getTask().handle(new TaskEvent(taskAttempt.getId().getTaskId(),
+ TaskEventType.T_ATTEMPT_KILLED));
+ LOG.info(taskAttempt.getId() + " Received TA_KILLED Status from LocalTask");
+ }
+ }
+
+ private static class StatusUpdateTransition
+ implements MultipleArcTransition<TaskAttempt, TaskAttemptEvent, TaskAttemptState> {
+
+ @Override
+ public TaskAttemptState transition(TaskAttempt taskAttempt,
+ TaskAttemptEvent event) {
+ TaskAttemptStatusUpdateEvent updateEvent = (TaskAttemptStatusUpdateEvent) event;
+
+ taskAttempt.progress = updateEvent.getStatus().getProgress();
+ taskAttempt.inputStats = updateEvent.getStatus().getInputStats();
+ taskAttempt.resultStats = updateEvent.getStatus().getResultStats();
+
+ return TaskAttemptState.TA_RUNNING;
+ }
+ }
+
+ private void addDiagnosticInfo(String diag) {
+ if (diag != null && !diag.equals("")) {
+ diagnostics.add(diag);
+ }
+ }
+
+ private static class AlreadyAssignedTransition
+ implements SingleArcTransition<TaskAttempt, TaskAttemptEvent>{
+
+ @Override
+ public void transition(TaskAttempt taskAttempt,
+ TaskAttemptEvent taskAttemptEvent) {
+ }
+ }
+
+ private static class AlreadyDoneTransition
+ implements SingleArcTransition<TaskAttempt, TaskAttemptEvent>{
+
+ @Override
+ public void transition(TaskAttempt taskAttempt,
+ TaskAttemptEvent taskAttemptEvent) {
+ }
+ }
+
+ private static class SucceededTransition implements SingleArcTransition<TaskAttempt, TaskAttemptEvent> {
+ @Override
+ public void transition(TaskAttempt taskAttempt,
+ TaskAttemptEvent event) {
+ TaskCompletionReport report = ((TaskCompletionEvent)event).getReport();
+
+ try {
+ taskAttempt.fillTaskStatistics(report);
+ taskAttempt.eventHandler.handle(new TaskTAttemptEvent(taskAttempt.getId(), TaskEventType.T_ATTEMPT_SUCCEEDED));
+ } catch (Throwable t) {
+ taskAttempt.eventHandler.handle(new TaskFatalErrorEvent(taskAttempt.getId(), t.getMessage()));
+ taskAttempt.addDiagnosticInfo(ExceptionUtils.getStackTrace(t));
+ }
+ }
+ }
+
+ private static class KillTaskTransition implements SingleArcTransition<TaskAttempt, TaskAttemptEvent> {
+
+ @Override
+ public void transition(TaskAttempt taskAttempt, TaskAttemptEvent event) {
+ taskAttempt.eventHandler.handle(new LocalTaskEvent(taskAttempt.getId(), taskAttempt.containerId,
+ LocalTaskEventType.KILL));
+ }
+ }
+
+ private static class FailedTransition implements SingleArcTransition<TaskAttempt, TaskAttemptEvent>{
+ @Override
+ public void transition(TaskAttempt taskAttempt, TaskAttemptEvent event) {
+ TaskFatalErrorEvent errorEvent = (TaskFatalErrorEvent) event;
+ taskAttempt.eventHandler.handle(new TaskTAttemptEvent(taskAttempt.getId(), TaskEventType.T_ATTEMPT_FAILED));
+ taskAttempt.addDiagnosticInfo(errorEvent.errorMessage());
+ LOG.error(taskAttempt.getId() + " FROM " + taskAttempt.getWorkerConnectionInfo().getHost()
+ + " >> " + errorEvent.errorMessage());
+ }
+ }
+
+ @Override
+ public void handle(TaskAttemptEvent event) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Processing " + event.getTaskAttemptId() + " of type " + event.getType());
+ }
+ try {
+ writeLock.lock();
+ TaskAttemptState oldState = getState();
+ try {
+ stateMachine.doTransition(event.getType(), event);
+ } catch (InvalidStateTransitonException e) {
+ LOG.error("Can't handle this event at current state of " + event.getTaskAttemptId() + ")"
+ + ", eventType:" + event.getType().name()
+ + ", oldState:" + oldState.name()
+ + ", nextState:" + getState().name()
+ , e);
+ eventHandler.handle(
+ new StageDiagnosticsUpdateEvent(event.getTaskAttemptId().getTaskId().getExecutionBlockId(),
+ "Can't handle this event at current state of " + event.getTaskAttemptId() + ")"));
+ eventHandler.handle(
+ new StageEvent(event.getTaskAttemptId().getTaskId().getExecutionBlockId(),
+ StageEventType.SQ_INTERNAL_ERROR));
+ }
+
+ //notify the eventhandler of state change
+ if (LOG.isDebugEnabled()) {
+ if (oldState != getState()) {
+ LOG.debug(id + " TaskAttempt Transitioned from " + oldState + " to "
+ + getState());
+ }
+ }
+ }
+
+ finally {
+ writeLock.unlock();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskSchedulerContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskSchedulerContext.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskSchedulerContext.java
new file mode 100644
index 0000000..b699674
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskSchedulerContext.java
@@ -0,0 +1,65 @@
+/*
+ * 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.tajo.querymaster;
+
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.querymaster.QueryMasterTask;
+
+public class TaskSchedulerContext {
+ private QueryMasterTask.QueryMasterTaskContext masterContext;
+ private boolean isLeafQuery;
+ private ExecutionBlockId blockId;
+ private int taskSize;
+ private int estimatedTaskNum;
+
+ public TaskSchedulerContext(QueryMasterTask.QueryMasterTaskContext masterContext, boolean isLeafQuery,
+ ExecutionBlockId blockId) {
+ this.masterContext = masterContext;
+ this.isLeafQuery = isLeafQuery;
+ this.blockId = blockId;
+ }
+
+ public QueryMasterTask.QueryMasterTaskContext getMasterContext() {
+ return masterContext;
+ }
+
+ public boolean isLeafQuery() {
+ return isLeafQuery;
+ }
+
+ public ExecutionBlockId getBlockId() {
+ return blockId;
+ }
+
+ public int getTaskSize() {
+ return taskSize;
+ }
+
+ public int getEstimatedTaskNum() {
+ return estimatedTaskNum;
+ }
+
+ public void setTaskSize(int taskSize) {
+ this.taskSize = taskSize;
+ }
+
+ public void setEstimatedTaskNum(int estimatedTaskNum) {
+ this.estimatedTaskNum = estimatedTaskNum;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskSchedulerFactory.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskSchedulerFactory.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskSchedulerFactory.java
new file mode 100644
index 0000000..2794771
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/TaskSchedulerFactory.java
@@ -0,0 +1,68 @@
+/*
+ * 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.tajo.querymaster;
+
+import com.google.common.collect.Maps;
+import org.apache.hadoop.conf.Configuration;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.util.Map;
+
+public class TaskSchedulerFactory {
+ private static Class<? extends AbstractTaskScheduler> CACHED_ALGORITHM_CLASS;
+ private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE = Maps.newConcurrentMap();
+ private static final Class<?>[] DEFAULT_PARAMS = { TaskSchedulerContext.class, Stage.class };
+
+ public static Class<? extends AbstractTaskScheduler> getTaskSchedulerClass(Configuration conf)
+ throws IOException {
+ if (CACHED_ALGORITHM_CLASS != null) {
+ return CACHED_ALGORITHM_CLASS;
+ } else {
+ CACHED_ALGORITHM_CLASS = conf.getClass("tajo.querymaster.task-scheduler", null, AbstractTaskScheduler.class);
+ }
+
+ if (CACHED_ALGORITHM_CLASS == null) {
+ throw new IOException("Task scheduler is null");
+ }
+ return CACHED_ALGORITHM_CLASS;
+ }
+
+ public static <T extends AbstractTaskScheduler> T get(Class<T> clazz, TaskSchedulerContext context,
+ Stage stage) {
+ T result;
+ try {
+ Constructor<T> constructor = (Constructor<T>) CONSTRUCTOR_CACHE.get(clazz);
+ if (constructor == null) {
+ constructor = clazz.getDeclaredConstructor(DEFAULT_PARAMS);
+ constructor.setAccessible(true);
+ CONSTRUCTOR_CACHE.put(clazz, constructor);
+ }
+ result = constructor.newInstance(new Object[]{context, stage});
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ return result;
+ }
+
+ public static AbstractTaskScheduler get(Configuration conf, TaskSchedulerContext context, Stage stage)
+ throws IOException {
+ return get(getTaskSchedulerClass(conf), context, stage);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/scheduler/QuerySchedulingInfo.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/scheduler/QuerySchedulingInfo.java b/tajo-core/src/main/java/org/apache/tajo/scheduler/QuerySchedulingInfo.java
deleted file mode 100644
index d9932bd..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/scheduler/QuerySchedulingInfo.java
+++ /dev/null
@@ -1,55 +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.tajo.scheduler;
-
-import com.google.common.base.Objects;
-import org.apache.tajo.QueryId;
-
-public class QuerySchedulingInfo {
- private QueryId queryId;
- private Integer priority;
- private Long startTime;
-
- public QuerySchedulingInfo(QueryId queryId, Integer priority, Long startTime) {
- this.queryId = queryId;
- this.priority = priority;
- this.startTime = startTime;
- }
-
- public QueryId getQueryId() {
- return queryId;
- }
-
- public Integer getPriority() {
- return priority;
- }
-
- public Long getStartTime() {
- return startTime;
- }
-
- public String getName() {
- return queryId.getId();
- }
-
- @Override
- public int hashCode() {
- return Objects.hashCode(startTime, getName(), priority);
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/scheduler/Scheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/scheduler/Scheduler.java b/tajo-core/src/main/java/org/apache/tajo/scheduler/Scheduler.java
deleted file mode 100644
index d74280c..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/scheduler/Scheduler.java
+++ /dev/null
@@ -1,41 +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.tajo.scheduler;
-
-import org.apache.tajo.QueryId;
-import org.apache.tajo.master.querymaster.QueryInProgress;
-
-import java.util.List;
-
-public interface Scheduler {
-
- public Mode getMode();
-
- public String getName();
-
- public boolean addQuery(QueryInProgress resource);
-
- public boolean removeQuery(QueryId queryId);
-
- public List<QueryInProgress> getRunningQueries();
-
- public enum Mode {
- FIFO
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/scheduler/SchedulingAlgorithms.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/scheduler/SchedulingAlgorithms.java b/tajo-core/src/main/java/org/apache/tajo/scheduler/SchedulingAlgorithms.java
deleted file mode 100644
index 9c9b16d..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/scheduler/SchedulingAlgorithms.java
+++ /dev/null
@@ -1,47 +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.tajo.scheduler;
-
-import java.util.Comparator;
-
-/**
- * Utility class containing scheduling algorithms used in the scheduler.
- */
-
-public class SchedulingAlgorithms {
- /**
- * Compare Schedulables in order of priority and then submission time, as in
- * the default FIFO scheduler in Tajo.
- */
- public static class FifoComparator implements Comparator<QuerySchedulingInfo> {
- @Override
- public int compare(QuerySchedulingInfo q1, QuerySchedulingInfo q2) {
- int res = q1.getPriority().compareTo(q2.getPriority());
- if (res == 0) {
- res = (int) Math.signum(q1.getStartTime() - q2.getStartTime());
- }
- if (res == 0) {
- // In the rare case where jobs were submitted at the exact same time,
- // compare them by name (which will be the QueryId) to get a deterministic ordering
- res = q1.getName().compareTo(q2.getName());
- }
- return res;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/scheduler/SimpleFifoScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/scheduler/SimpleFifoScheduler.java b/tajo-core/src/main/java/org/apache/tajo/scheduler/SimpleFifoScheduler.java
deleted file mode 100644
index a74e606..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/scheduler/SimpleFifoScheduler.java
+++ /dev/null
@@ -1,147 +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.tajo.scheduler;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.master.querymaster.QueryInProgress;
-import org.apache.tajo.master.querymaster.QueryJobManager;
-
-import java.util.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-public class SimpleFifoScheduler implements Scheduler {
- private static final Log LOG = LogFactory.getLog(SimpleFifoScheduler.class.getName());
- private LinkedList<QuerySchedulingInfo> pool = new LinkedList<QuerySchedulingInfo>();
- private final Thread queryProcessor;
- private AtomicBoolean stopped = new AtomicBoolean();
- private QueryJobManager manager;
- private Comparator<QuerySchedulingInfo> COMPARATOR = new SchedulingAlgorithms.FifoComparator();
-
- public SimpleFifoScheduler(QueryJobManager manager) {
- this.manager = manager;
- this.queryProcessor = new Thread(new QueryProcessor());
- this.queryProcessor.setName("Query Processor");
- }
-
- @Override
- public Mode getMode() {
- return Mode.FIFO;
- }
-
- @Override
- public String getName() {
- return manager.getName();
- }
-
- @Override
- public boolean addQuery(QueryInProgress queryInProgress) {
- int qSize = pool.size();
- if (qSize != 0 && qSize % 100 == 0) {
- LOG.info("Size of Fifo queue is " + qSize);
- }
-
- QuerySchedulingInfo querySchedulingInfo = new QuerySchedulingInfo(queryInProgress.getQueryId(), 1, queryInProgress.getStartTime());
- boolean result = pool.add(querySchedulingInfo);
- if (getRunningQueries().size() == 0) wakeupProcessor();
- return result;
- }
-
- @Override
- public boolean removeQuery(QueryId queryId) {
- return pool.remove(getQueryByQueryId(queryId));
- }
-
- public QuerySchedulingInfo getQueryByQueryId(QueryId queryId) {
- for (QuerySchedulingInfo querySchedulingInfo : pool) {
- if (querySchedulingInfo.getQueryId().equals(queryId)) {
- return querySchedulingInfo;
- }
- }
- return null;
- }
-
- @Override
- public List<QueryInProgress> getRunningQueries() {
- return new ArrayList<QueryInProgress>(manager.getRunningQueries());
- }
-
- public void start() {
- queryProcessor.start();
- }
-
- public void stop() {
- if (stopped.getAndSet(true)) {
- return;
- }
- pool.clear();
- synchronized (queryProcessor) {
- queryProcessor.interrupt();
- }
- }
-
- private QuerySchedulingInfo pollScheduledQuery() {
- if (pool.size() > 1) {
- Collections.sort(pool, COMPARATOR);
- }
- return pool.poll();
- }
-
- private void wakeupProcessor() {
- synchronized (queryProcessor) {
- queryProcessor.notifyAll();
- }
- }
-
- private final class QueryProcessor implements Runnable {
- @Override
- public void run() {
-
- QuerySchedulingInfo query;
-
- while (!stopped.get() && !Thread.currentThread().isInterrupted()) {
- query = null;
- if (getRunningQueries().size() == 0) {
- query = pollScheduledQuery();
- }
-
- if (query != null) {
- try {
- manager.startQueryJob(query.getQueryId());
- } catch (Throwable t) {
- LOG.fatal("Exception during query startup:", t);
- manager.stopQuery(query.getQueryId());
- }
- }
-
- synchronized (queryProcessor) {
- try {
- queryProcessor.wait(500);
- } catch (InterruptedException e) {
- if (stopped.get()) {
- break;
- }
- LOG.warn("Exception during shutdown: ", e);
- }
- }
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/session/InvalidSessionException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/session/InvalidSessionException.java b/tajo-core/src/main/java/org/apache/tajo/session/InvalidSessionException.java
new file mode 100644
index 0000000..54c65bf
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/session/InvalidSessionException.java
@@ -0,0 +1,25 @@
+/**
+ * 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.tajo.session;
+
+public class InvalidSessionException extends Exception {
+ public InvalidSessionException(String sessionId) {
+ super("Invalid session id \"" + sessionId + "\"");
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/session/NoSuchSessionVariableException.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/session/NoSuchSessionVariableException.java b/tajo-core/src/main/java/org/apache/tajo/session/NoSuchSessionVariableException.java
new file mode 100644
index 0000000..be90449
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/session/NoSuchSessionVariableException.java
@@ -0,0 +1,25 @@
+/**
+ * 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.tajo.session;
+
+public class NoSuchSessionVariableException extends Exception {
+ public NoSuchSessionVariableException(String varname) {
+ super("No such session variable \"" + varname + "\"");
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/session/Session.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/session/Session.java b/tajo-core/src/main/java/org/apache/tajo/session/Session.java
new file mode 100644
index 0000000..7ac4f85
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/session/Session.java
@@ -0,0 +1,196 @@
+/**
+ * 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.tajo.session;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.SessionVars;
+import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
+import org.apache.tajo.util.KeyValueSet;
+import org.apache.tajo.common.ProtoObject;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.tajo.ipc.TajoWorkerProtocol.SessionProto;
+
+public class Session implements SessionConstants, ProtoObject<SessionProto>, Cloneable {
+ private static final Log LOG = LogFactory.getLog(Session.class);
+
+ private final String sessionId;
+ private final String userName;
+ private String currentDatabase;
+ private final Map<String, String> sessionVariables;
+ private final Map<QueryId, NonForwardQueryResultScanner> nonForwardQueryMap = new HashMap<QueryId, NonForwardQueryResultScanner>();
+
+ // transient status
+ private volatile long lastAccessTime;
+
+ public Session(String sessionId, String userName, String databaseName) {
+ this.sessionId = sessionId;
+ this.userName = userName;
+ this.currentDatabase = databaseName;
+ this.lastAccessTime = System.currentTimeMillis();
+
+ this.sessionVariables = new HashMap<String, String>();
+ sessionVariables.put(SessionVars.SESSION_ID.keyname(), sessionId);
+ sessionVariables.put(SessionVars.USERNAME.keyname(), userName);
+ selectDatabase(databaseName);
+ }
+
+ public Session(SessionProto proto) {
+ sessionId = proto.getSessionId();
+ userName = proto.getUsername();
+ currentDatabase = proto.getCurrentDatabase();
+ lastAccessTime = proto.getLastAccessTime();
+ KeyValueSet keyValueSet = new KeyValueSet(proto.getVariables());
+ sessionVariables = keyValueSet.getAllKeyValus();
+ }
+
+ public String getSessionId() {
+ return sessionId;
+ }
+
+ public String getUserName() {
+ return userName;
+ }
+
+ public void updateLastAccessTime() {
+ lastAccessTime = System.currentTimeMillis();
+ }
+
+ public long getLastAccessTime() {
+ return lastAccessTime;
+ }
+
+ public void setVariable(String name, String value) {
+ synchronized (sessionVariables) {
+ sessionVariables.put(SessionVars.handleDeprecatedName(name), value);
+ }
+ }
+
+ public String getVariable(String name) throws NoSuchSessionVariableException {
+ synchronized (sessionVariables) {
+ if (sessionVariables.containsKey(name)) {
+ return sessionVariables.get(SessionVars.handleDeprecatedName(name));
+ } else {
+ throw new NoSuchSessionVariableException(name);
+ }
+ }
+ }
+
+ public void removeVariable(String name) {
+ synchronized (sessionVariables) {
+ sessionVariables.remove(SessionVars.handleDeprecatedName(name));
+ }
+ }
+
+ public synchronized Map<String, String> getAllVariables() {
+ synchronized (sessionVariables) {
+ sessionVariables.put(SessionVars.SESSION_ID.keyname(), sessionId);
+ sessionVariables.put(SessionVars.USERNAME.keyname(), userName);
+ sessionVariables.put(SessionVars.SESSION_LAST_ACCESS_TIME.keyname(), String.valueOf(lastAccessTime));
+ sessionVariables.put(SessionVars.CURRENT_DATABASE.keyname(), currentDatabase);
+ return ImmutableMap.copyOf(sessionVariables);
+ }
+ }
+
+ public synchronized void selectDatabase(String databaseName) {
+ this.currentDatabase = databaseName;
+ }
+
+ public synchronized String getCurrentDatabase() {
+ return currentDatabase;
+ }
+
+ @Override
+ public SessionProto getProto() {
+ SessionProto.Builder builder = SessionProto.newBuilder();
+ builder.setSessionId(getSessionId());
+ builder.setUsername(getUserName());
+ builder.setCurrentDatabase(getCurrentDatabase());
+ builder.setLastAccessTime(lastAccessTime);
+ KeyValueSet variables = new KeyValueSet();
+
+ synchronized (sessionVariables) {
+ variables.putAll(this.sessionVariables);
+ builder.setVariables(variables.getProto());
+ return builder.build();
+ }
+ }
+
+ public String toString() {
+ return "user=" + getUserName() + ",id=" + getSessionId() +",last_atime=" + getLastAccessTime();
+ }
+
+ public Session clone() throws CloneNotSupportedException {
+ Session newSession = (Session) super.clone();
+ newSession.sessionVariables.putAll(getAllVariables());
+ return newSession;
+ }
+
+ public NonForwardQueryResultScanner getNonForwardQueryResultScanner(QueryId queryId) {
+ synchronized (nonForwardQueryMap) {
+ return nonForwardQueryMap.get(queryId);
+ }
+ }
+
+ public void addNonForwardQueryResultScanner(NonForwardQueryResultScanner resultScanner) {
+ synchronized (nonForwardQueryMap) {
+ nonForwardQueryMap.put(resultScanner.getQueryId(), resultScanner);
+ }
+ }
+
+ public void closeNonForwardQueryResultScanner(QueryId queryId) {
+ NonForwardQueryResultScanner resultScanner;
+ synchronized (nonForwardQueryMap) {
+ resultScanner = nonForwardQueryMap.remove(queryId);
+ }
+
+ if (resultScanner != null) {
+ try {
+ resultScanner.close();
+ } catch (Exception e) {
+ LOG.error("NonForwardQueryResultScanne close error: " + e.getMessage(), e);
+ }
+ }
+ }
+
+ public void close() {
+ try {
+ synchronized (nonForwardQueryMap) {
+ for (NonForwardQueryResultScanner eachQueryScanner: nonForwardQueryMap.values()) {
+ try {
+ eachQueryScanner.close();
+ } catch (Exception e) {
+ LOG.error("Error while closing NonForwardQueryResultScanner: " +
+ eachQueryScanner.getSessionId() + ", " + e.getMessage(), e);
+ }
+ }
+
+ nonForwardQueryMap.clear();
+ }
+ } catch (Throwable t) {
+ LOG.error(t.getMessage(), t);
+ throw new RuntimeException(t.getMessage(), t);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/session/SessionConstants.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/session/SessionConstants.java b/tajo-core/src/main/java/org/apache/tajo/session/SessionConstants.java
new file mode 100644
index 0000000..6c21a27
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/session/SessionConstants.java
@@ -0,0 +1,23 @@
+/**
+ * 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.tajo.session;
+
+public interface SessionConstants {
+
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/session/SessionEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/session/SessionEvent.java b/tajo-core/src/main/java/org/apache/tajo/session/SessionEvent.java
new file mode 100644
index 0000000..819fd16
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/session/SessionEvent.java
@@ -0,0 +1,34 @@
+/**
+ * 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.tajo.session;
+
+import org.apache.hadoop.yarn.event.AbstractEvent;
+
+public class SessionEvent extends AbstractEvent<SessionEventType> {
+ private final String sessionId;
+
+ public SessionEvent(String sessionId, SessionEventType sessionEventType) {
+ super(sessionEventType);
+ this.sessionId = sessionId;
+ }
+
+ public String getSessionId() {
+ return sessionId;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/session/SessionEventType.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/session/SessionEventType.java b/tajo-core/src/main/java/org/apache/tajo/session/SessionEventType.java
new file mode 100644
index 0000000..8270926
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/session/SessionEventType.java
@@ -0,0 +1,24 @@
+/**
+ * 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.tajo.session;
+
+public enum SessionEventType {
+ EXPIRE,
+ PING
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/session/SessionLivelinessMonitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/session/SessionLivelinessMonitor.java b/tajo-core/src/main/java/org/apache/tajo/session/SessionLivelinessMonitor.java
new file mode 100644
index 0000000..2badccb
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/session/SessionLivelinessMonitor.java
@@ -0,0 +1,53 @@
+/**
+ * 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.tajo.session;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.AbstractLivelinessMonitor;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tajo.conf.TajoConf;
+
+public class SessionLivelinessMonitor extends AbstractLivelinessMonitor<String> {
+
+ private EventHandler dispatcher;
+
+ public SessionLivelinessMonitor(Dispatcher d) {
+ super(SessionLivelinessMonitor.class.getSimpleName(), new SystemClock());
+ this.dispatcher = d.getEventHandler();
+ }
+
+ public void serviceInit(Configuration conf) throws Exception {
+ Preconditions.checkArgument(conf instanceof TajoConf);
+ TajoConf systemConf = (TajoConf) conf;
+
+ // seconds
+ int expireIntvl = systemConf.getIntVar(TajoConf.ConfVars.$CLIENT_SESSION_EXPIRY_TIME);
+ setExpireInterval(expireIntvl);
+ setMonitorInterval(expireIntvl / 3);
+ super.serviceInit(conf);
+ }
+
+ @Override
+ protected void expire(String id) {
+ dispatcher.handle(new SessionEvent(id, SessionEventType.EXPIRE));
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/session/SessionManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/session/SessionManager.java b/tajo-core/src/main/java/org/apache/tajo/session/SessionManager.java
new file mode 100644
index 0000000..571144b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/session/SessionManager.java
@@ -0,0 +1,144 @@
+/**
+ * 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.tajo.session;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.event.Dispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class SessionManager extends CompositeService implements EventHandler<SessionEvent> {
+ private static final Log LOG = LogFactory.getLog(SessionManager.class);
+
+ public final ConcurrentHashMap<String, Session> sessions = new ConcurrentHashMap<String, Session>();
+ private final Dispatcher dispatcher;
+ private SessionLivelinessMonitor sessionLivelinessMonitor;
+
+
+ public SessionManager(Dispatcher dispatcher) {
+ super(SessionManager.class.getSimpleName());
+ this.dispatcher = dispatcher;
+ }
+
+ @Override
+ public void serviceInit(Configuration conf) throws Exception {
+ sessionLivelinessMonitor = new SessionLivelinessMonitor(dispatcher);
+ addIfService(sessionLivelinessMonitor);
+ super.serviceInit(conf);
+ }
+
+ @Override
+ public void serviceStop() throws Exception {
+ super.serviceStop();
+ }
+
+ private void assertSessionExistence(String sessionId) throws InvalidSessionException {
+ if (!sessions.containsKey(sessionId)) {
+ throw new InvalidSessionException(sessionId);
+ }
+ }
+
+ public String createSession(String username, String baseDatabaseName) throws InvalidSessionException {
+ String sessionId;
+ Session oldSession;
+
+ sessionId = UUID.randomUUID().toString();
+ Session newSession = new Session(sessionId, username, baseDatabaseName);
+ oldSession = sessions.putIfAbsent(sessionId, newSession);
+ if (oldSession != null) {
+ throw new InvalidSessionException("Session id is duplicated: " + oldSession.getSessionId());
+ }
+ LOG.info("Session " + sessionId + " is created." );
+ return sessionId;
+ }
+
+ public Session removeSession(String sessionId) {
+ if (sessions.containsKey(sessionId)) {
+ LOG.info("Session " + sessionId + " is removed.");
+ Session session = sessions.remove(sessionId);
+ session.close();
+ return session;
+ } else {
+ LOG.error("No such session id: " + sessionId);
+ return null;
+ }
+ }
+
+ public Session getSession(String sessionId) throws InvalidSessionException {
+ assertSessionExistence(sessionId);
+ touch(sessionId);
+ return sessions.get(sessionId);
+ }
+
+ public void setVariable(String sessionId, String name, String value) throws InvalidSessionException {
+ assertSessionExistence(sessionId);
+ touch(sessionId);
+ sessions.get(sessionId).setVariable(name, value);
+ }
+
+ public String getVariable(String sessionId, String name)
+ throws InvalidSessionException, NoSuchSessionVariableException {
+ assertSessionExistence(sessionId);
+ touch(sessionId);
+ return sessions.get(sessionId).getVariable(name);
+ }
+
+ public void removeVariable(String sessionId, String name) throws InvalidSessionException {
+ assertSessionExistence(sessionId);
+ touch(sessionId);
+ sessions.get(sessionId).removeVariable(name);
+ }
+
+ public Map<String, String> getAllVariables(String sessionId) throws InvalidSessionException {
+ assertSessionExistence(sessionId);
+ touch(sessionId);
+ return sessions.get(sessionId).getAllVariables();
+ }
+
+ public void touch(String sessionId) throws InvalidSessionException {
+ assertSessionExistence(sessionId);
+ sessions.get(sessionId).updateLastAccessTime();
+ sessionLivelinessMonitor.receivedPing(sessionId);
+ }
+
+ @Override
+ public void handle(SessionEvent event) {
+ LOG.info("Processing " + event.getSessionId() + " of type " + event.getType());
+
+ try {
+ assertSessionExistence(event.getSessionId());
+ touch(event.getSessionId());
+ } catch (InvalidSessionException e) {
+ LOG.error(e);
+ }
+
+ if (event.getType() == SessionEventType.EXPIRE) {
+ Session session = removeSession(event.getSessionId());
+ if (session != null) {
+ LOG.info("[Expired] Session username=" + session.getUserName() + ",sessionid=" + event.getSessionId());
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java b/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java
index 6050617..d711258 100644
--- a/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java
+++ b/tajo-core/src/main/java/org/apache/tajo/util/JSPUtil.java
@@ -24,11 +24,11 @@ import org.apache.tajo.catalog.proto.CatalogProtos;
import org.apache.tajo.catalog.statistics.TableStats;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.master.TajoMaster.MasterContext;
-import org.apache.tajo.master.ha.HAService;
-import org.apache.tajo.master.querymaster.QueryInProgress;
-import org.apache.tajo.master.querymaster.QueryMasterTask;
-import org.apache.tajo.master.querymaster.Task;
-import org.apache.tajo.master.querymaster.Stage;
+import org.apache.tajo.ha.HAService;
+import org.apache.tajo.querymaster.QueryInProgress;
+import org.apache.tajo.querymaster.QueryMasterTask;
+import org.apache.tajo.querymaster.Task;
+import org.apache.tajo.querymaster.Stage;
import org.apache.tajo.util.history.TaskHistory;
import org.apache.tajo.util.history.StageHistory;
import org.apache.tajo.worker.TaskRunnerHistory;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java
index 932f584..c3f0087 100644
--- a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java
+++ b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryReader.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.tajo.TaskAttemptId;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.ipc.TajoWorkerProtocol.TaskHistoryProto;
-import org.apache.tajo.master.querymaster.QueryInfo;
+import org.apache.tajo.master.QueryInfo;
import java.io.EOFException;
import java.io.IOException;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java
index 5934885..9eb58da 100644
--- a/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java
+++ b/tajo-core/src/main/java/org/apache/tajo/util/history/HistoryWriter.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.*;
import org.apache.hadoop.service.AbstractService;
import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.master.querymaster.QueryInfo;
+import org.apache.tajo.master.QueryInfo;
import org.apache.tajo.worker.TaskHistory;
import java.io.IOException;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java b/tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java
index 208591f..89c3404 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/FetchImpl.java
@@ -24,8 +24,8 @@ import com.google.common.collect.Lists;
import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.common.ProtoObject;
import org.apache.tajo.ipc.TajoWorkerProtocol;
-import org.apache.tajo.master.querymaster.Task;
-import org.apache.tajo.master.querymaster.Repartitioner;
+import org.apache.tajo.querymaster.Task;
+import org.apache.tajo.querymaster.Repartitioner;
import org.apache.tajo.util.TUtil;
import java.net.URI;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
index 8944eae..8241478 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoResourceAllocator.java
@@ -38,9 +38,9 @@ import org.apache.tajo.master.container.TajoContainerId;
import org.apache.tajo.master.event.ContainerAllocationEvent;
import org.apache.tajo.master.event.ContainerAllocatorEventType;
import org.apache.tajo.master.event.StageContainerAllocationEvent;
-import org.apache.tajo.master.querymaster.QueryMasterTask;
-import org.apache.tajo.master.querymaster.Stage;
-import org.apache.tajo.master.querymaster.StageState;
+import org.apache.tajo.querymaster.QueryMasterTask;
+import org.apache.tajo.querymaster.Stage;
+import org.apache.tajo.querymaster.StageState;
import org.apache.tajo.master.rm.*;
import org.apache.tajo.rpc.CallFuture;
import org.apache.tajo.rpc.NettyClientBase;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
index 4d96529..09a87e0 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
@@ -38,9 +38,9 @@ import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.ha.HAServiceUtil;
import org.apache.tajo.ipc.TajoMasterProtocol;
import org.apache.tajo.master.cluster.WorkerConnectionInfo;
-import org.apache.tajo.master.ha.TajoMasterInfo;
-import org.apache.tajo.master.querymaster.QueryMaster;
-import org.apache.tajo.master.querymaster.QueryMasterManagerService;
+import org.apache.tajo.ha.TajoMasterInfo;
+import org.apache.tajo.querymaster.QueryMaster;
+import org.apache.tajo.querymaster.QueryMasterManagerService;
import org.apache.tajo.master.rm.TajoWorkerResourceManager;
import org.apache.tajo.pullserver.TajoPullServerService;
import org.apache.tajo.rpc.RpcChannelFactory;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
index 1c83110..2ae4bed 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorkerClientService.java
@@ -31,7 +31,7 @@ import org.apache.tajo.ipc.ClientProtos.GetQueryHistoryResponse;
import org.apache.tajo.ipc.ClientProtos.QueryIdRequest;
import org.apache.tajo.ipc.ClientProtos.ResultCode;
import org.apache.tajo.ipc.QueryMasterClientProtocol;
-import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.querymaster.QueryMasterTask;
import org.apache.tajo.rpc.BlockingRpcServer;
import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
import org.apache.tajo.util.NetUtils;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/resources/tajo-default.xml
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/tajo-default.xml b/tajo-core/src/main/resources/tajo-default.xml
index db92b02..4a92e72 100644
--- a/tajo-core/src/main/resources/tajo-default.xml
+++ b/tajo-core/src/main/resources/tajo-default.xml
@@ -39,7 +39,7 @@
<property>
<name>tajo.querymaster.task-scheduler</name>
- <value>org.apache.tajo.master.DefaultTaskScheduler</value>
+ <value>org.apache.tajo.querymaster.DefaultTaskScheduler</value>
</property>
</configuration>
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/resources/webapps/admin/catalogview.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/catalogview.jsp b/tajo-core/src/main/resources/webapps/admin/catalogview.jsp
index 8f1d1bc..bc770d7 100644
--- a/tajo-core/src/main/resources/webapps/admin/catalogview.jsp
+++ b/tajo-core/src/main/resources/webapps/admin/catalogview.jsp
@@ -24,7 +24,7 @@
<%@ page import="org.apache.tajo.catalog.TableDesc" %>
<%@ page import="org.apache.tajo.catalog.partition.PartitionMethodDesc" %>
<%@ page import="org.apache.tajo.master.TajoMaster" %>
-<%@ page import="org.apache.tajo.master.ha.HAService" %>
+<%@ page import="org.apache.tajo.ha.HAService" %>
<%@ page import="org.apache.tajo.util.FileUtil" %>
<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
<%@ page import="java.util.Collection" %>
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/resources/webapps/admin/cluster.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/cluster.jsp b/tajo-core/src/main/resources/webapps/admin/cluster.jsp
index 6fe21a2..1fb5e40 100644
--- a/tajo-core/src/main/resources/webapps/admin/cluster.jsp
+++ b/tajo-core/src/main/resources/webapps/admin/cluster.jsp
@@ -21,8 +21,8 @@
<%@ page import="org.apache.tajo.master.TajoMaster" %>
<%@ page import="org.apache.tajo.master.cluster.WorkerConnectionInfo" %>
-<%@ page import="org.apache.tajo.master.ha.HAService" %>
-<%@ page import="org.apache.tajo.master.ha.TajoMasterInfo" %>
+<%@ page import="org.apache.tajo.ha.HAService" %>
+<%@ page import="org.apache.tajo.ha.TajoMasterInfo" %>
<%@ page import="org.apache.tajo.master.rm.Worker" %>
<%@ page import="org.apache.tajo.master.rm.WorkerResource" %>
<%@ page import="org.apache.tajo.master.rm.WorkerState" %>
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/resources/webapps/admin/index.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/index.jsp b/tajo-core/src/main/resources/webapps/admin/index.jsp
index 6778725..00186d7 100644
--- a/tajo-core/src/main/resources/webapps/admin/index.jsp
+++ b/tajo-core/src/main/resources/webapps/admin/index.jsp
@@ -23,9 +23,9 @@
<%@ page import="org.apache.tajo.conf.TajoConf" %>
<%@ page import="org.apache.tajo.ipc.TajoMasterProtocol" %>
<%@ page import="org.apache.tajo.master.TajoMaster" %>
-<%@ page import="org.apache.tajo.master.ha.HAService" %>
-<%@ page import="org.apache.tajo.master.ha.TajoMasterInfo" %>
-<%@ page import="org.apache.tajo.master.querymaster.QueryInProgress" %>
+<%@ page import="org.apache.tajo.ha.HAService" %>
+<%@ page import="org.apache.tajo.ha.TajoMasterInfo" %>
+<%@ page import="org.apache.tajo.querymaster.QueryInProgress" %>
<%@ page import="org.apache.tajo.master.rm.Worker" %>
<%@ page import="org.apache.tajo.master.rm.WorkerState" %>
<%@ page import="org.apache.tajo.util.NetUtils" %>
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/resources/webapps/admin/query.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/query.jsp b/tajo-core/src/main/resources/webapps/admin/query.jsp
index 5afb3b2..4d8e5e6 100644
--- a/tajo-core/src/main/resources/webapps/admin/query.jsp
+++ b/tajo-core/src/main/resources/webapps/admin/query.jsp
@@ -20,7 +20,7 @@
<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
<%@ page import="org.apache.tajo.master.TajoMaster" %>
-<%@ page import="org.apache.tajo.master.querymaster.QueryInProgress" %>
+<%@ page import="org.apache.tajo.querymaster.QueryInProgress" %>
<%@ page import="org.apache.tajo.master.rm.Worker" %>
<%@ page import="org.apache.tajo.util.JSPUtil" %>
<%@ page import="org.apache.tajo.util.StringUtils" %>
@@ -28,7 +28,7 @@
<%@ page import="java.text.SimpleDateFormat" %>
<%@ page import="java.util.*" %>
<%@ page import="org.apache.tajo.util.history.HistoryReader" %>
-<%@ page import="org.apache.tajo.master.querymaster.QueryInfo" %>
+<%@ page import="org.apache.tajo.master.QueryInfo" %>
<%
TajoMaster master = (TajoMaster) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/resources/webapps/admin/query_executor.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/admin/query_executor.jsp b/tajo-core/src/main/resources/webapps/admin/query_executor.jsp
index 9ff6625..82836ac 100644
--- a/tajo-core/src/main/resources/webapps/admin/query_executor.jsp
+++ b/tajo-core/src/main/resources/webapps/admin/query_executor.jsp
@@ -19,7 +19,7 @@
%>
<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
<%@ page import="org.apache.tajo.master.TajoMaster" %>
-<%@ page import="org.apache.tajo.master.ha.HAService" %>
+<%@ page import="org.apache.tajo.ha.HAService" %>
<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
<%
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/resources/webapps/worker/index.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/index.jsp b/tajo-core/src/main/resources/webapps/worker/index.jsp
index 866d663..bb72f9e 100644
--- a/tajo-core/src/main/resources/webapps/worker/index.jsp
+++ b/tajo-core/src/main/resources/webapps/worker/index.jsp
@@ -19,8 +19,8 @@
%>
<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
-<%@ page import="org.apache.tajo.master.querymaster.Query" %>
-<%@ page import="org.apache.tajo.master.querymaster.QueryMasterTask" %>
+<%@ page import="org.apache.tajo.querymaster.Query" %>
+<%@ page import="org.apache.tajo.querymaster.QueryMasterTask" %>
<%@ page import="org.apache.tajo.util.JSPUtil" %>
<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
<%@ page import="org.apache.tajo.worker.TajoWorker" %>
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/resources/webapps/worker/querydetail.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/querydetail.jsp b/tajo-core/src/main/resources/webapps/worker/querydetail.jsp
index 340eb95..56bdeba 100644
--- a/tajo-core/src/main/resources/webapps/worker/querydetail.jsp
+++ b/tajo-core/src/main/resources/webapps/worker/querydetail.jsp
@@ -20,8 +20,8 @@
<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
<%@ page import="org.apache.tajo.QueryId" %>
-<%@ page import="org.apache.tajo.master.querymaster.Query" %>
-<%@ page import="org.apache.tajo.master.querymaster.QueryMasterTask" %>
+<%@ page import="org.apache.tajo.querymaster.Query" %>
+<%@ page import="org.apache.tajo.querymaster.QueryMasterTask" %>
<%@ page import="org.apache.tajo.util.JSPUtil" %>
<%@ page import="org.apache.tajo.util.TajoIdUtils" %>
<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/resources/webapps/worker/queryplan.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/queryplan.jsp b/tajo-core/src/main/resources/webapps/worker/queryplan.jsp
index 88de97d..878efe3 100644
--- a/tajo-core/src/main/resources/webapps/worker/queryplan.jsp
+++ b/tajo-core/src/main/resources/webapps/worker/queryplan.jsp
@@ -21,11 +21,11 @@
<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
<%@ page import="org.apache.tajo.worker.*" %>
-<%@ page import="org.apache.tajo.master.querymaster.Query" %>
+<%@ page import="org.apache.tajo.querymaster.Query" %>
<%@ page import="org.apache.tajo.QueryId" %>
<%@ page import="org.apache.tajo.util.TajoIdUtils" %>
-<%@ page import="org.apache.tajo.master.querymaster.QueryMasterTask" %>
-<%@ page import="org.apache.tajo.master.querymaster.Stage" %>
+<%@ page import="org.apache.tajo.querymaster.QueryMasterTask" %>
+<%@ page import="org.apache.tajo.querymaster.Stage" %>
<%@ page import="org.apache.tajo.engine.planner.global.ExecutionBlock" %>
<%@ page import="java.util.*" %>
<%@ page import="org.apache.tajo.ExecutionBlockId" %>
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/resources/webapps/worker/querytasks.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/querytasks.jsp b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp
index 3aef49d..6d0e3a2 100644
--- a/tajo-core/src/main/resources/webapps/worker/querytasks.jsp
+++ b/tajo-core/src/main/resources/webapps/worker/querytasks.jsp
@@ -25,7 +25,7 @@
<%@ page import="org.apache.tajo.catalog.statistics.TableStats" %>
<%@ page import="org.apache.tajo.plan.util.PlannerUtil" %>
<%@ page import="org.apache.tajo.ipc.TajoMasterProtocol" %>
-<%@ page import="org.apache.tajo.master.querymaster.*" %>
+<%@ page import="org.apache.tajo.querymaster.*" %>
<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
<%@ page import="org.apache.tajo.worker.TajoWorker" %>
<%@ page import="java.text.NumberFormat" %>
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/resources/webapps/worker/task.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/resources/webapps/worker/task.jsp b/tajo-core/src/main/resources/webapps/worker/task.jsp
index 81b1e6d..17e884a 100644
--- a/tajo-core/src/main/resources/webapps/worker/task.jsp
+++ b/tajo-core/src/main/resources/webapps/worker/task.jsp
@@ -25,10 +25,10 @@
<%@ page import="org.apache.tajo.catalog.proto.CatalogProtos" %>
<%@ page import="org.apache.tajo.catalog.statistics.TableStats" %>
<%@ page import="org.apache.tajo.ipc.TajoWorkerProtocol" %>
-<%@ page import="org.apache.tajo.master.querymaster.Query" %>
-<%@ page import="org.apache.tajo.master.querymaster.QueryMasterTask" %>
-<%@ page import="org.apache.tajo.master.querymaster.Task" %>
-<%@ page import="org.apache.tajo.master.querymaster.Stage" %>
+<%@ page import="org.apache.tajo.querymaster.Query" %>
+<%@ page import="org.apache.tajo.querymaster.QueryMasterTask" %>
+<%@ page import="org.apache.tajo.querymaster.Task" %>
+<%@ page import="org.apache.tajo.querymaster.Stage" %>
<%@ page import="org.apache.tajo.storage.DataLocation" %>
<%@ page import="org.apache.tajo.storage.fragment.FileFragment" %>
<%@ page import="org.apache.tajo.storage.fragment.FragmentConvertor" %>
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java b/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
index 8bee6fb..e464446 100644
--- a/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
+++ b/tajo-core/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
@@ -35,7 +35,7 @@ import org.apache.tajo.client.TajoClientImpl;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.engine.planner.global.MasterPlan;
import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.master.session.Session;
+import org.apache.tajo.session.Session;
import org.apache.tajo.util.CommonTestingUtil;
import org.apache.tajo.util.FileUtil;
import org.apache.tajo.util.KeyValueSet;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
index 0d2f6fa..0786912 100644
--- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
+++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java
@@ -44,10 +44,10 @@ import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.conf.TajoConf.ConfVars;
import org.apache.tajo.engine.planner.global.rewriter.GlobalPlanTestRuleProvider;
import org.apache.tajo.master.TajoMaster;
-import org.apache.tajo.master.querymaster.*;
-import org.apache.tajo.master.querymaster.Query;
-import org.apache.tajo.master.querymaster.Stage;
-import org.apache.tajo.master.querymaster.StageState;
+import org.apache.tajo.querymaster.*;
+import org.apache.tajo.querymaster.Query;
+import org.apache.tajo.querymaster.Stage;
+import org.apache.tajo.querymaster.StageState;
import org.apache.tajo.master.rm.TajoWorkerResourceManager;
import org.apache.tajo.plan.rewrite.LogicalPlanTestRuleProvider;
import org.apache.tajo.util.CommonTestingUtil;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
index 889d61c..0b59bc7 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
@@ -37,7 +37,7 @@ import org.apache.tajo.engine.function.builtin.SumInt;
import org.apache.tajo.engine.json.CoreGsonHelper;
import org.apache.tajo.engine.parser.SQLAnalyzer;
import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.master.session.Session;
+import org.apache.tajo.session.Session;
import org.apache.tajo.plan.*;
import org.apache.tajo.plan.expr.*;
import org.apache.tajo.plan.logical.*;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
index 6db76ae..d3ab1fd 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
@@ -44,7 +44,7 @@ import org.apache.tajo.engine.planner.enforce.Enforcer;
import org.apache.tajo.engine.planner.global.DataChannel;
import org.apache.tajo.engine.planner.global.MasterPlan;
import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.master.session.Session;
+import org.apache.tajo.session.Session;
import org.apache.tajo.plan.LogicalOptimizer;
import org.apache.tajo.plan.LogicalPlan;
import org.apache.tajo.plan.LogicalPlanner;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
index 1a212b0..d1756e1 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
@@ -24,10 +24,10 @@ import org.apache.tajo.*;
import org.apache.tajo.catalog.Schema;
import org.apache.tajo.common.TajoDataTypes.Type;
import org.apache.tajo.conf.TajoConf.ConfVars;
-import org.apache.tajo.master.querymaster.Query;
-import org.apache.tajo.master.querymaster.QueryMasterTask;
-import org.apache.tajo.master.querymaster.Task;
-import org.apache.tajo.master.querymaster.Stage;
+import org.apache.tajo.querymaster.Query;
+import org.apache.tajo.querymaster.QueryMasterTask;
+import org.apache.tajo.querymaster.Task;
+import org.apache.tajo.querymaster.Stage;
import org.apache.tajo.storage.StorageConstants;
import org.apache.tajo.util.KeyValueSet;
import org.apache.tajo.util.TUtil;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
index 68b3fb3..39b58d0 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
@@ -32,7 +32,7 @@ import org.apache.tajo.engine.planner.global.ExecutionBlock;
import org.apache.tajo.engine.planner.global.MasterPlan;
import org.apache.tajo.plan.logical.NodeType;
import org.apache.tajo.jdbc.TajoResultSet;
-import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.querymaster.QueryMasterTask;
import org.apache.tajo.storage.*;
import org.apache.tajo.util.FileUtil;
import org.apache.tajo.util.KeyValueSet;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
index 3400752..cacef96 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
@@ -38,7 +38,7 @@ import org.apache.tajo.engine.planner.global.ExecutionBlock;
import org.apache.tajo.engine.planner.global.MasterPlan;
import org.apache.tajo.ipc.ClientProtos;
import org.apache.tajo.jdbc.TajoResultSet;
-import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.querymaster.QueryMasterTask;
import org.apache.tajo.plan.logical.NodeType;
import org.apache.tajo.storage.StorageConstants;
import org.apache.tajo.util.CommonTestingUtil;
[13/16] tajo git commit: TAJO-1288: Refactoring
org.apache.tajo.master package.
Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultSystemScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultSystemScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultSystemScanner.java
deleted file mode 100644
index c6466f5..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultSystemScanner.java
+++ /dev/null
@@ -1,616 +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.tajo.master;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Stack;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.TaskAttemptId;
-import org.apache.tajo.TaskId;
-import org.apache.tajo.catalog.CatalogUtil;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.catalog.TableMeta;
-import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.DatabaseProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.IndexProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableDescriptorProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableOptionProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TablePartitionProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableStatsProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TablespaceProto;
-import org.apache.tajo.catalog.statistics.TableStats;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.engine.codegen.CompilationError;
-import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
-import org.apache.tajo.engine.planner.Projector;
-import org.apache.tajo.engine.planner.global.ExecutionBlock;
-import org.apache.tajo.engine.planner.global.ExecutionBlockCursor;
-import org.apache.tajo.engine.planner.global.GlobalPlanner;
-import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.engine.planner.physical.PhysicalExec;
-import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.master.TajoMaster.MasterContext;
-import org.apache.tajo.plan.LogicalPlan;
-import org.apache.tajo.plan.PlanningException;
-import org.apache.tajo.plan.expr.EvalNode;
-import org.apache.tajo.plan.logical.IndexScanNode;
-import org.apache.tajo.plan.logical.LogicalNode;
-import org.apache.tajo.plan.logical.ScanNode;
-import org.apache.tajo.storage.RowStoreUtil;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.VTuple;
-import org.apache.tajo.storage.RowStoreUtil.RowStoreEncoder;
-import org.apache.tajo.util.KeyValueSet;
-import org.apache.tajo.util.TUtil;
-import org.apache.tajo.worker.TaskAttemptContext;
-
-import com.google.protobuf.ByteString;
-
-public class NonForwardQueryResultSystemScanner implements NonForwardQueryResultScanner {
-
- private final Log LOG = LogFactory.getLog(getClass());
-
- private MasterContext masterContext;
- private LogicalPlan logicalPlan;
- private final QueryId queryId;
- private final String sessionId;
- private TaskAttemptContext taskContext;
- private int currentRow;
- private long maxRow;
- private TableDesc tableDesc;
- private Schema outSchema;
- private RowStoreEncoder encoder;
- private PhysicalExec physicalExec;
-
- public NonForwardQueryResultSystemScanner(MasterContext context, LogicalPlan plan, QueryId queryId,
- String sessionId, int maxRow) {
- masterContext = context;
- logicalPlan = plan;
- this.queryId = queryId;
- this.sessionId = sessionId;
- this.maxRow = maxRow;
-
- }
-
- @Override
- public void init() throws IOException {
- QueryContext queryContext = new QueryContext(masterContext.getConf());
- currentRow = 0;
-
- MasterPlan masterPlan = new MasterPlan(queryId, queryContext, logicalPlan);
- GlobalPlanner globalPlanner = new GlobalPlanner(masterContext.getConf(), masterContext.getCatalog());
- try {
- globalPlanner.build(masterPlan);
- } catch (PlanningException e) {
- throw new RuntimeException(e);
- }
-
- ExecutionBlockCursor cursor = new ExecutionBlockCursor(masterPlan);
- ExecutionBlock leafBlock = null;
- while (cursor.hasNext()) {
- ExecutionBlock block = cursor.nextBlock();
- if (masterPlan.isLeaf(block)) {
- leafBlock = block;
- break;
- }
- }
-
- taskContext = new TaskAttemptContext(queryContext, null,
- new TaskAttemptId(new TaskId(leafBlock.getId(), 0), 0),
- null, null);
- physicalExec = new SimplePhysicalPlannerImpl(masterContext.getConf())
- .createPlan(taskContext, leafBlock.getPlan());
-
- tableDesc = new TableDesc("table_"+System.currentTimeMillis(), physicalExec.getSchema(),
- new TableMeta(StoreType.SYSTEM, new KeyValueSet()), null);
- outSchema = physicalExec.getSchema();
- encoder = RowStoreUtil.createEncoder(getLogicalSchema());
-
- physicalExec.init();
- }
-
- @Override
- public void close() throws Exception {
- tableDesc = null;
- outSchema = null;
- encoder = null;
- if (physicalExec != null) {
- try {
- physicalExec.close();
- } catch (Exception ignored) {}
- }
- physicalExec = null;
- currentRow = -1;
- }
-
- private List<Tuple> getTablespaces(Schema outSchema) {
- List<TablespaceProto> tablespaces = masterContext.getCatalog().getAllTablespaces();
- List<Tuple> tuples = new ArrayList<Tuple>(tablespaces.size());
- List<Column> columns = outSchema.getColumns();
- Tuple aTuple;
-
- for (TablespaceProto tablespace: tablespaces) {
- aTuple = new VTuple(outSchema.size());
-
- for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
- Column column = columns.get(fieldId);
- if ("space_id".equalsIgnoreCase(column.getSimpleName())) {
- if (tablespace.hasId()) {
- aTuple.put(fieldId, DatumFactory.createInt4(tablespace.getId()));
- } else {
- aTuple.put(fieldId, DatumFactory.createNullDatum());
- }
- } else if ("space_name".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(tablespace.getSpaceName()));
- } else if ("space_handler".equalsIgnoreCase(column.getSimpleName())) {
- if (tablespace.hasHandler()) {
- aTuple.put(fieldId, DatumFactory.createText(tablespace.getHandler()));
- } else {
- aTuple.put(fieldId, DatumFactory.createNullDatum());
- }
- } else if ("space_uri".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(tablespace.getUri()));
- }
- }
- tuples.add(aTuple);
- }
-
- return tuples;
- }
-
- private List<Tuple> getDatabases(Schema outSchema) {
- List<DatabaseProto> databases = masterContext.getCatalog().getAllDatabases();
- List<Tuple> tuples = new ArrayList<Tuple>(databases.size());
- List<Column> columns = outSchema.getColumns();
- Tuple aTuple;
-
- for (DatabaseProto database: databases) {
- aTuple = new VTuple(outSchema.size());
-
- for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
- Column column = columns.get(fieldId);
- if ("db_id".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt4(database.getId()));
- } else if ("db_name".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(database.getName()));
- } else if ("space_id".equalsIgnoreCase(column.getSimpleName())) {
- if (database.hasSpaceId()) {
- aTuple.put(fieldId, DatumFactory.createInt4(database.getSpaceId()));
- } else {
- aTuple.put(fieldId, DatumFactory.createNullDatum());
- }
- }
- }
-
- tuples.add(aTuple);
- }
-
- return tuples;
- }
-
- private List<Tuple> getTables(Schema outSchema) {
- List<TableDescriptorProto> tables = masterContext.getCatalog().getAllTables();
- List<Tuple> tuples = new ArrayList<Tuple>(tables.size());
- List<Column> columns = outSchema.getColumns();
- Tuple aTuple;
-
- for (TableDescriptorProto table: tables) {
- aTuple = new VTuple(outSchema.size());
-
- for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
- Column column = columns.get(fieldId);
- if ("tid".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt4(table.getTid()));
- } else if ("db_id".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt4(table.getDbId()));
- } else if ("table_name".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(table.getName()));
- } else if ("table_type".equalsIgnoreCase(column.getSimpleName())) {
- if (table.hasTableType()) {
- aTuple.put(fieldId, DatumFactory.createText(table.getTableType()));
- } else {
- aTuple.put(fieldId, DatumFactory.createNullDatum());
- }
- } else if ("path".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(table.getPath()));
- } else if ("store_type".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(table.getStoreType()));
- }
- }
-
- tuples.add(aTuple);
- }
-
- return tuples;
- }
-
- private List<Tuple> getColumns(Schema outSchema) {
- List<ColumnProto> columnsList = masterContext.getCatalog().getAllColumns();
- List<Tuple> tuples = new ArrayList<Tuple>(columnsList.size());
- List<Column> columns = outSchema.getColumns();
- Tuple aTuple;
- int columnId = 1, prevtid = -1, tid = 0;
-
- for (ColumnProto column: columnsList) {
- aTuple = new VTuple(outSchema.size());
-
- tid = column.getTid();
- if (prevtid != tid) {
- columnId = 1;
- prevtid = tid;
- }
-
- for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
- Column colObj = columns.get(fieldId);
-
- if ("tid".equalsIgnoreCase(colObj.getSimpleName())) {
- if (column.hasTid()) {
- aTuple.put(fieldId, DatumFactory.createInt4(tid));
- } else {
- aTuple.put(fieldId, DatumFactory.createNullDatum());
- }
- } else if ("column_name".equalsIgnoreCase(colObj.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(column.getName()));
- } else if ("ordinal_position".equalsIgnoreCase(colObj.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt4(columnId));
- } else if ("data_type".equalsIgnoreCase(colObj.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(column.getDataType().getType().toString()));
- } else if ("type_length".equalsIgnoreCase(colObj.getSimpleName())) {
- DataType dataType = column.getDataType();
- if (dataType.hasLength()) {
- aTuple.put(fieldId, DatumFactory.createInt4(dataType.getLength()));
- } else {
- aTuple.put(fieldId, DatumFactory.createNullDatum());
- }
- }
- }
-
- columnId++;
- tuples.add(aTuple);
- }
-
- return tuples;
- }
-
- private List<Tuple> getIndexes(Schema outSchema) {
- List<IndexProto> indexList = masterContext.getCatalog().getAllIndexes();
- List<Tuple> tuples = new ArrayList<Tuple>(indexList.size());
- List<Column> columns = outSchema.getColumns();
- Tuple aTuple;
-
- for (IndexProto index: indexList) {
- aTuple = new VTuple(outSchema.size());
-
- for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
- Column column = columns.get(fieldId);
-
- if ("db_id".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt4(index.getDbId()));
- } else if ("tid".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt4(index.getTId()));
- } else if ("index_name".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(index.getIndexName()));
- } else if ("column_name".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(index.getColumnName()));
- } else if ("data_type".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(index.getDataType()));
- } else if ("index_type".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(index.getIndexType()));
- } else if ("is_unique".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createBool(index.getIsUnique()));
- } else if ("is_clustered".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createBool(index.getIsClustered()));
- } else if ("is_ascending".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createBool(index.getIsAscending()));
- }
- }
-
- tuples.add(aTuple);
- }
-
- return tuples;
- }
-
- private List<Tuple> getAllTableOptions(Schema outSchema) {
- List<TableOptionProto> optionList = masterContext.getCatalog().getAllTableOptions();
- List<Tuple> tuples = new ArrayList<Tuple>(optionList.size());
- List<Column> columns = outSchema.getColumns();
- Tuple aTuple;
-
- for (TableOptionProto option: optionList) {
- aTuple = new VTuple(outSchema.size());
-
- for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
- Column column = columns.get(fieldId);
-
- if ("tid".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt4(option.getTid()));
- } else if ("key_".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(option.getKeyval().getKey()));
- } else if ("value_".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(option.getKeyval().getValue()));
- }
- }
-
- tuples.add(aTuple);
- }
-
- return tuples;
- }
-
- private List<Tuple> getAllTableStats(Schema outSchema) {
- List<TableStatsProto> statList = masterContext.getCatalog().getAllTableStats();
- List<Tuple> tuples = new ArrayList<Tuple>(statList.size());
- List<Column> columns = outSchema.getColumns();
- Tuple aTuple;
-
- for (TableStatsProto stat: statList) {
- aTuple = new VTuple(outSchema.size());
-
- for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
- Column column = columns.get(fieldId);
-
- if ("tid".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt4(stat.getTid()));
- } else if ("num_rows".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt8(stat.getNumRows()));
- } else if ("num_bytes".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt8(stat.getNumBytes()));
- }
- }
-
- tuples.add(aTuple);
- }
-
- return tuples;
- }
-
- private List<Tuple> getAllPartitions(Schema outSchema) {
- List<TablePartitionProto> partitionList = masterContext.getCatalog().getAllPartitions();
- List<Tuple> tuples = new ArrayList<Tuple>(partitionList.size());
- List<Column> columns = outSchema.getColumns();
- Tuple aTuple;
-
- for (TablePartitionProto partition: partitionList) {
- aTuple = new VTuple(outSchema.size());
-
- for (int fieldId = 0; fieldId < columns.size(); fieldId++) {
- Column column = columns.get(fieldId);
-
- if ("pid".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt4(partition.getPid()));
- } else if ("tid".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt4(partition.getTid()));
- } else if ("partition_name".equalsIgnoreCase(column.getSimpleName())) {
- if (partition.hasPartitionName()) {
- aTuple.put(fieldId, DatumFactory.createText(partition.getPartitionName()));
- } else {
- aTuple.put(fieldId, DatumFactory.createNullDatum());
- }
- } else if ("ordinal_position".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createInt4(partition.getOrdinalPosition()));
- } else if ("path".equalsIgnoreCase(column.getSimpleName())) {
- aTuple.put(fieldId, DatumFactory.createText(partition.getPath()));
- }
- }
-
- tuples.add(aTuple);
- }
-
- return tuples;
- }
-
- private List<Tuple> fetchSystemTable(TableDesc tableDesc, Schema inSchema) {
- List<Tuple> tuples = null;
- String tableName = CatalogUtil.extractSimpleName(tableDesc.getName());
-
- if ("tablespace".equalsIgnoreCase(tableName)) {
- tuples = getTablespaces(inSchema);
- } else if ("databases".equalsIgnoreCase(tableName)) {
- tuples = getDatabases(inSchema);
- } else if ("tables".equalsIgnoreCase(tableName)) {
- tuples = getTables(inSchema);
- } else if ("columns".equalsIgnoreCase(tableName)) {
- tuples = getColumns(inSchema);
- } else if ("indexes".equalsIgnoreCase(tableName)) {
- tuples = getIndexes(inSchema);
- } else if ("table_options".equalsIgnoreCase(tableName)) {
- tuples = getAllTableOptions(inSchema);
- } else if ("table_stats".equalsIgnoreCase(tableName)) {
- tuples = getAllTableStats(inSchema);
- } else if ("partitions".equalsIgnoreCase(tableName)) {
- tuples = getAllPartitions(inSchema);
- }
-
- return tuples;
- }
-
- @Override
- public List<ByteString> getNextRows(int fetchRowNum) throws IOException {
- List<ByteString> rows = new ArrayList<ByteString>();
- int startRow = currentRow;
- int endRow = startRow + fetchRowNum;
-
- if (physicalExec == null) {
- return rows;
- }
-
- while (currentRow < endRow) {
- Tuple currentTuple = physicalExec.next();
-
- if (currentTuple == null) {
- physicalExec.close();
- physicalExec = null;
- break;
- }
-
- currentRow++;
- rows.add(ByteString.copyFrom(encoder.toBytes(currentTuple)));
-
- if (currentRow >= maxRow) {
- physicalExec.close();
- physicalExec = null;
- break;
- }
- }
-
- return rows;
- }
-
- @Override
- public QueryId getQueryId() {
- return queryId;
- }
-
- @Override
- public String getSessionId() {
- return sessionId;
- }
-
- @Override
- public TableDesc getTableDesc() {
- return tableDesc;
- }
-
- @Override
- public Schema getLogicalSchema() {
- return outSchema;
- }
-
- class SimplePhysicalPlannerImpl extends PhysicalPlannerImpl {
-
- public SimplePhysicalPlannerImpl(TajoConf conf) {
- super(conf);
- }
-
- @Override
- public PhysicalExec createScanPlan(TaskAttemptContext ctx, ScanNode scanNode, Stack<LogicalNode> node)
- throws IOException {
- return new SystemPhysicalExec(ctx, scanNode);
- }
-
- @Override
- public PhysicalExec createIndexScanExec(TaskAttemptContext ctx, IndexScanNode annotation) throws IOException {
- return new SystemPhysicalExec(ctx, annotation);
- }
- }
-
- class SystemPhysicalExec extends PhysicalExec {
-
- private ScanNode scanNode;
- private EvalNode qual;
- private Projector projector;
- private TableStats tableStats;
- private final List<Tuple> cachedData;
- private int currentRow;
- private boolean isClosed;
-
- public SystemPhysicalExec(TaskAttemptContext context, ScanNode scanNode) {
- super(context, scanNode.getInSchema(), scanNode.getOutSchema());
- this.scanNode = scanNode;
- this.qual = this.scanNode.getQual();
- cachedData = TUtil.newList();
- currentRow = 0;
- isClosed = false;
-
- projector = new Projector(context, inSchema, outSchema, scanNode.getTargets());
- }
-
- @Override
- public Tuple next() throws IOException {
- Tuple aTuple = null;
- Tuple outTuple = new VTuple(outColumnNum);
-
- if (isClosed) {
- return null;
- }
-
- if (cachedData.size() == 0) {
- rescan();
- }
-
- if (!scanNode.hasQual()) {
- if (currentRow < cachedData.size()) {
- aTuple = cachedData.get(currentRow++);
- projector.eval(aTuple, outTuple);
- outTuple.setOffset(aTuple.getOffset());
- return outTuple;
- }
- return null;
- } else {
- while (currentRow < cachedData.size()) {
- aTuple = cachedData.get(currentRow++);
- if (qual.eval(inSchema, aTuple).isTrue()) {
- projector.eval(aTuple, outTuple);
- return outTuple;
- }
- }
- return null;
- }
- }
-
- @Override
- public void rescan() throws IOException {
- cachedData.clear();
- cachedData.addAll(fetchSystemTable(scanNode.getTableDesc(), inSchema));
-
- tableStats = new TableStats();
- tableStats.setNumRows(cachedData.size());
- }
-
- @Override
- public void close() throws IOException {
- scanNode = null;
- qual = null;
- projector = null;
- cachedData.clear();
- currentRow = -1;
- isClosed = true;
- }
-
- @Override
- public float getProgress() {
- return 1.0f;
- }
-
- @Override
- protected void compile() throws CompilationError {
- if (scanNode.hasQual()) {
- qual = context.getPrecompiledEval(inSchema, qual);
- }
- }
-
- @Override
- public TableStats getInputStats() {
- return tableStats;
- }
-
- }
-
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/QueryInfo.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryInfo.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryInfo.java
new file mode 100644
index 0000000..f902081
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryInfo.java
@@ -0,0 +1,235 @@
+/**
+ * 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.tajo.master;
+
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.ipc.ClientProtos.QueryInfoProto;
+import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.util.TajoIdUtils;
+import org.apache.tajo.util.history.History;
+
+public class QueryInfo implements GsonObject, History {
+ private QueryId queryId;
+ @Expose
+ private QueryContext context;
+ @Expose
+ private String sql;
+ @Expose
+ private volatile TajoProtos.QueryState queryState;
+ @Expose
+ private volatile float progress;
+ @Expose
+ private volatile long startTime;
+ @Expose
+ private volatile long finishTime;
+ @Expose
+ private String lastMessage;
+ @Expose
+ private String hostNameOfQM;
+ @Expose
+ private int queryMasterPort;
+ @Expose
+ private int queryMasterClientPort;
+ @Expose
+ private int queryMasterInfoPort;
+ @Expose
+ private String queryIdStr;
+ @Expose
+ private volatile TableDesc resultDesc;
+
+ private String jsonExpr;
+
+ public QueryInfo(QueryId queryId) {
+ this(queryId, null, null, null);
+ }
+
+ public QueryInfo(QueryId queryId, QueryContext queryContext, String sql, String jsonExpr) {
+ this.queryId = queryId;
+ this.queryIdStr = queryId.toString();
+ this.context = queryContext;
+ this.sql = sql;
+ this.jsonExpr = jsonExpr;
+
+ this.queryState = TajoProtos.QueryState.QUERY_MASTER_INIT;
+ }
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public QueryContext getQueryContext() {
+ return context;
+ }
+
+ public String getSql() {
+ return sql;
+ }
+
+ public String getQueryMasterHost() {
+ return hostNameOfQM;
+ }
+
+ public void setQueryMaster(String hostName) {
+ this.hostNameOfQM = hostName;
+ }
+
+ public int getQueryMasterInfoPort() {
+ return queryMasterInfoPort;
+ }
+
+ public void setQueryMasterInfoPort(int queryMasterInfoPort) {
+ this.queryMasterInfoPort = queryMasterInfoPort;
+ }
+
+ public void setQueryMasterPort(int port) {
+ this.queryMasterPort = port;
+ }
+
+ public int getQueryMasterPort() {
+ return queryMasterPort;
+ }
+
+ public void setQueryMasterclientPort(int port) {
+ queryMasterClientPort = port;
+ }
+
+ public int getQueryMasterClientPort() {
+ return queryMasterClientPort;
+ }
+
+ public TajoProtos.QueryState getQueryState() {
+ return queryState;
+ }
+
+ public void setQueryState(TajoProtos.QueryState queryState) {
+ this.queryState = queryState;
+ }
+
+ public long getStartTime() {
+ return startTime;
+ }
+
+ public void setStartTime(long startTime) {
+ this.startTime = startTime;
+ }
+
+ public long getFinishTime() {
+ return finishTime;
+ }
+
+ public void setFinishTime(long finishTime) {
+ this.finishTime = finishTime;
+ }
+
+ public String getLastMessage() {
+ return lastMessage;
+ }
+
+ public void setLastMessage(String lastMessage) {
+ this.lastMessage = lastMessage;
+ }
+
+ public float getProgress() {
+ return progress;
+ }
+
+ public void setProgress(float progress) {
+ this.progress = progress;
+ }
+
+ public void setResultDesc(TableDesc result) {
+ this.resultDesc = result;
+ }
+
+ public boolean hasResultdesc() {
+ return resultDesc != null;
+ }
+
+ public TableDesc getResultDesc() {
+ return resultDesc;
+ }
+
+ @Override
+ public String toString() {
+ return queryId.toString() + ",state=" + queryState +",progress=" + progress + ", queryMaster="
+ + getQueryMasterHost();
+ }
+
+ public String getJsonExpr() {
+ return jsonExpr;
+ }
+
+ @Override
+ public String toJson() {
+ return CoreGsonHelper.toJson(this, QueryInfo.class);
+ }
+
+ @Override
+ public HistoryType getHistoryType() {
+ return HistoryType.QUERY_SUMMARY;
+ }
+
+ public static QueryInfo fromJson(String json) {
+ QueryInfo queryInfo = CoreGsonHelper.fromJson(json, QueryInfo.class);
+ queryInfo.queryId = TajoIdUtils.parseQueryId(queryInfo.queryIdStr);
+ return queryInfo;
+ }
+
+ public String getQueryIdStr() {
+ return queryIdStr;
+ }
+
+ public QueryInfoProto getProto() {
+ QueryInfoProto.Builder builder = QueryInfoProto.newBuilder();
+
+ builder.setQueryId(queryId.toString())
+ .setQueryState(queryState)
+ .setContextVars(context.getProto())
+ .setProgress(progress)
+ .setStartTime(startTime)
+ .setFinishTime(finishTime)
+ .setQueryMasterPort(queryMasterPort)
+ .setQueryMasterClientPort(queryMasterClientPort)
+ .setQueryMasterInfoPort(queryMasterInfoPort);
+
+ if (resultDesc != null) {
+ builder.setResultDesc(resultDesc.getProto());
+ }
+
+ if (sql != null) {
+ builder.setSql(sql);
+ }
+
+ if (lastMessage != null) {
+ builder.setLastMessage(lastMessage);
+ }
+
+ if (hostNameOfQM != null) {
+ builder.setHostNameOfQM(hostNameOfQM);
+ }
+
+ return builder.build();
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/QueryJobManager.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/QueryJobManager.java b/tajo-core/src/main/java/org/apache/tajo/master/QueryJobManager.java
new file mode 100644
index 0000000..c9b8711
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/QueryJobManager.java
@@ -0,0 +1,311 @@
+/**
+ * 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.tajo.master;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.TajoProtos;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.ipc.TajoMasterProtocol;
+import org.apache.tajo.master.cluster.WorkerConnectionInfo;
+import org.apache.tajo.querymaster.QueryInProgress;
+import org.apache.tajo.querymaster.QueryJobEvent;
+import org.apache.tajo.session.Session;
+import org.apache.tajo.plan.logical.LogicalRootNode;
+import org.apache.tajo.master.scheduler.SimpleFifoScheduler;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class QueryJobManager extends CompositeService {
+ private static final Log LOG = LogFactory.getLog(QueryJobManager.class.getName());
+
+ // TajoMaster Context
+ private final TajoMaster.MasterContext masterContext;
+
+ private AsyncDispatcher dispatcher;
+
+ private SimpleFifoScheduler scheduler;
+
+ private final Map<QueryId, QueryInProgress> submittedQueries = Maps.newConcurrentMap();
+
+ private final Map<QueryId, QueryInProgress> runningQueries = Maps.newConcurrentMap();
+
+ private AtomicLong minExecutionTime = new AtomicLong(Long.MAX_VALUE);
+ private AtomicLong maxExecutionTime = new AtomicLong();
+ private AtomicLong avgExecutionTime = new AtomicLong();
+ private AtomicLong executedQuerySize = new AtomicLong();
+
+ public QueryJobManager(final TajoMaster.MasterContext masterContext) {
+ super(QueryJobManager.class.getName());
+ this.masterContext = masterContext;
+ }
+
+ @Override
+ public void init(Configuration conf) {
+ try {
+ this.dispatcher = new AsyncDispatcher();
+ addService(this.dispatcher);
+
+ this.dispatcher.register(QueryJobEvent.Type.class, new QueryJobManagerEventHandler());
+
+ this.scheduler = new SimpleFifoScheduler(this);
+ } catch (Exception e) {
+ catchException(null, e);
+ }
+
+ super.init(conf);
+ }
+
+ @Override
+ public void stop() {
+ synchronized(runningQueries) {
+ for(QueryInProgress eachQueryInProgress: runningQueries.values()) {
+ eachQueryInProgress.stop();
+ }
+ }
+ this.scheduler.stop();
+ super.stop();
+ }
+
+ @Override
+ public void start() {
+ this.scheduler.start();
+ super.start();
+ }
+
+ public EventHandler getEventHandler() {
+ return dispatcher.getEventHandler();
+ }
+
+ public Collection<QueryInProgress> getSubmittedQueries() {
+ synchronized (submittedQueries){
+ return Collections.unmodifiableCollection(submittedQueries.values());
+ }
+ }
+
+ public Collection<QueryInProgress> getRunningQueries() {
+ synchronized (runningQueries){
+ return Collections.unmodifiableCollection(runningQueries.values());
+ }
+ }
+
+ public synchronized Collection<QueryInfo> getFinishedQueries() {
+ try {
+ return this.masterContext.getHistoryReader().getQueries(null);
+ } catch (Throwable e) {
+ LOG.error(e);
+ return Lists.newArrayList();
+ }
+ }
+
+
+ public synchronized QueryInfo getFinishedQuery(QueryId queryId) {
+ try {
+ return this.masterContext.getHistoryReader().getQueryInfo(queryId.toString());
+ } catch (Throwable e) {
+ LOG.error(e);
+ return null;
+ }
+ }
+
+ public QueryInfo createNewQueryJob(Session session, QueryContext queryContext, String sql,
+ String jsonExpr, LogicalRootNode plan)
+ throws Exception {
+ QueryId queryId = QueryIdFactory.newQueryId(masterContext.getResourceManager().getSeedQueryId());
+ QueryInProgress queryInProgress = new QueryInProgress(masterContext, session, queryContext, queryId, sql,
+ jsonExpr, plan);
+
+ synchronized (submittedQueries) {
+ queryInProgress.getQueryInfo().setQueryMaster("");
+ submittedQueries.put(queryInProgress.getQueryId(), queryInProgress);
+ }
+
+ scheduler.addQuery(queryInProgress);
+ return queryInProgress.getQueryInfo();
+ }
+
+ public QueryInfo startQueryJob(QueryId queryId) throws Exception {
+
+ QueryInProgress queryInProgress;
+
+ synchronized (submittedQueries) {
+ queryInProgress = submittedQueries.remove(queryId);
+ }
+
+ synchronized (runningQueries) {
+ runningQueries.put(queryInProgress.getQueryId(), queryInProgress);
+ }
+
+ addService(queryInProgress);
+ queryInProgress.init(getConfig());
+ queryInProgress.start();
+
+ if (!queryInProgress.startQueryMaster()) {
+ stopQuery(queryId);
+ }
+
+ return queryInProgress.getQueryInfo();
+ }
+
+ public TajoMaster.MasterContext getMasterContext() {
+ return masterContext;
+ }
+
+ class QueryJobManagerEventHandler implements EventHandler<QueryJobEvent> {
+ @Override
+ public void handle(QueryJobEvent event) {
+ QueryInProgress queryInProgress = getQueryInProgress(event.getQueryInfo().getQueryId());
+ if(queryInProgress == null) {
+ LOG.warn("No query info in running queries.[" + event.getQueryInfo().getQueryId() + "]");
+ return;
+ }
+
+ if (event.getType() == QueryJobEvent.Type.QUERY_JOB_STOP) {
+ stopQuery(event.getQueryInfo().getQueryId());
+ } else if (queryInProgress.isStarted()) {
+ queryInProgress.getEventHandler().handle(event);
+ } else if (event.getType() == QueryJobEvent.Type.QUERY_JOB_KILL) {
+ scheduler.removeQuery(queryInProgress.getQueryId());
+ queryInProgress.getQueryInfo().setQueryState(TajoProtos.QueryState.QUERY_KILLED);
+
+ stopQuery(queryInProgress.getQueryId());
+ }
+ }
+ }
+
+ public QueryInProgress getQueryInProgress(QueryId queryId) {
+ QueryInProgress queryInProgress;
+ synchronized (submittedQueries) {
+ queryInProgress = submittedQueries.get(queryId);
+ }
+
+ if (queryInProgress == null) {
+ synchronized (runningQueries) {
+ queryInProgress = runningQueries.get(queryId);
+ }
+ }
+ return queryInProgress;
+ }
+
+ public void stopQuery(QueryId queryId) {
+ LOG.info("Stop QueryInProgress:" + queryId);
+ QueryInProgress queryInProgress = getQueryInProgress(queryId);
+ if(queryInProgress != null) {
+ queryInProgress.stop();
+ synchronized(submittedQueries) {
+ submittedQueries.remove(queryId);
+ }
+
+ synchronized(runningQueries) {
+ runningQueries.remove(queryId);
+ }
+
+ QueryInfo queryInfo = queryInProgress.getQueryInfo();
+ long executionTime = queryInfo.getFinishTime() - queryInfo.getStartTime();
+ if (executionTime < minExecutionTime.get()) {
+ minExecutionTime.set(executionTime);
+ }
+
+ if (executionTime > maxExecutionTime.get()) {
+ maxExecutionTime.set(executionTime);
+ }
+
+ long totalExecutionTime = executedQuerySize.get() * avgExecutionTime.get();
+ if (totalExecutionTime > 0) {
+ avgExecutionTime.set((totalExecutionTime + executionTime) / (executedQuerySize.get() + 1));
+ } else {
+ avgExecutionTime.set(executionTime);
+ }
+ executedQuerySize.incrementAndGet();
+ removeService(queryInProgress);
+ } else {
+ LOG.warn("No QueryInProgress while query stopping: " + queryId);
+ }
+ }
+
+ public long getMinExecutionTime() {
+ if (getExecutedQuerySize() == 0) return 0;
+ return minExecutionTime.get();
+ }
+
+ public long getMaxExecutionTime() {
+ return maxExecutionTime.get();
+ }
+
+ public long getAvgExecutionTime() {
+ return avgExecutionTime.get();
+ }
+
+ public long getExecutedQuerySize() {
+ return executedQuerySize.get();
+ }
+
+ private void catchException(QueryId queryId, Exception e) {
+ LOG.error(e.getMessage(), e);
+ QueryInProgress queryInProgress = runningQueries.get(queryId);
+ queryInProgress.catchException(e);
+ }
+
+ public synchronized TajoMasterProtocol.TajoHeartbeatResponse.ResponseCommand queryHeartbeat(
+ TajoMasterProtocol.TajoHeartbeat queryHeartbeat) {
+ QueryInProgress queryInProgress = getQueryInProgress(new QueryId(queryHeartbeat.getQueryId()));
+ if(queryInProgress == null) {
+ return null;
+ }
+
+ QueryInfo queryInfo = makeQueryInfoFromHeartbeat(queryHeartbeat);
+ getEventHandler().handle(new QueryJobEvent(QueryJobEvent.Type.QUERY_JOB_HEARTBEAT, queryInfo));
+
+ return null;
+ }
+
+ private QueryInfo makeQueryInfoFromHeartbeat(TajoMasterProtocol.TajoHeartbeat queryHeartbeat) {
+ QueryInfo queryInfo = new QueryInfo(new QueryId(queryHeartbeat.getQueryId()));
+ WorkerConnectionInfo connectionInfo = new WorkerConnectionInfo(queryHeartbeat.getConnectionInfo());
+
+ queryInfo.setQueryMaster(connectionInfo.getHost());
+ queryInfo.setQueryMasterPort(connectionInfo.getQueryMasterPort());
+ queryInfo.setQueryMasterclientPort(connectionInfo.getClientPort());
+ queryInfo.setLastMessage(queryHeartbeat.getStatusMessage());
+ queryInfo.setQueryState(queryHeartbeat.getState());
+ queryInfo.setProgress(queryHeartbeat.getQueryProgress());
+
+ if (queryHeartbeat.hasQueryFinishTime()) {
+ queryInfo.setFinishTime(queryHeartbeat.getQueryFinishTime());
+ }
+
+ if (queryHeartbeat.hasResultDesc()) {
+ queryInfo.setResultDesc(new TableDesc(queryHeartbeat.getResultDesc()));
+ }
+
+ return queryInfo;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/ScheduledFetches.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/ScheduledFetches.java b/tajo-core/src/main/java/org/apache/tajo/master/ScheduledFetches.java
deleted file mode 100644
index b05572b..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/ScheduledFetches.java
+++ /dev/null
@@ -1,49 +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.tajo.master;
-
-import org.apache.tajo.worker.FetchImpl;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-public class ScheduledFetches {
- private List<Map<String, List<FetchImpl>>> fetches = new ArrayList<Map<String, List<FetchImpl>>>();
-
- public void addFetch(Map<String, List<FetchImpl>> fetch) {
- this.fetches.add(fetch);
- }
-
- public boolean hasNextFetch() {
- return fetches.size() > 0;
- }
-
- public Map<String, List<FetchImpl>> getNextFetch() {
- return hasNextFetch() ? fetches.get(0) : null;
- }
-
- public Map<String, List<FetchImpl>> popNextFetch() {
- return hasNextFetch() ? fetches.remove(0) : null;
- }
-
- public int size() {
- return fetches.size();
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
index 4649d99..7209080 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoContainerProxy.java
@@ -28,7 +28,7 @@ import org.apache.tajo.engine.query.QueryContext;
import org.apache.tajo.ipc.ContainerProtocol;
import org.apache.tajo.ipc.TajoMasterProtocol;
import org.apache.tajo.ipc.TajoWorkerProtocol;
-import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.querymaster.QueryMasterTask;
import org.apache.tajo.master.container.TajoContainer;
import org.apache.tajo.master.container.TajoContainerId;
import org.apache.tajo.master.rm.TajoWorkerContainer;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
index d021e43..c054599 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -40,14 +40,13 @@ import org.apache.tajo.catalog.LocalCatalogWrapper;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.conf.TajoConf.ConfVars;
import org.apache.tajo.engine.function.FunctionLoader;
-import org.apache.tajo.master.ha.HAService;
-import org.apache.tajo.master.ha.HAServiceHDFSImpl;
-import org.apache.tajo.master.metrics.CatalogMetricsGaugeSet;
-import org.apache.tajo.master.metrics.WorkerResourceMetricsGaugeSet;
-import org.apache.tajo.master.querymaster.QueryJobManager;
+import org.apache.tajo.ha.HAService;
+import org.apache.tajo.ha.HAServiceHDFSImpl;
+import org.apache.tajo.metrics.CatalogMetricsGaugeSet;
+import org.apache.tajo.metrics.WorkerResourceMetricsGaugeSet;
import org.apache.tajo.master.rm.TajoWorkerResourceManager;
import org.apache.tajo.master.rm.WorkerResourceManager;
-import org.apache.tajo.master.session.SessionManager;
+import org.apache.tajo.session.SessionManager;
import org.apache.tajo.rpc.RpcChannelFactory;
import org.apache.tajo.rule.EvaluationContext;
import org.apache.tajo.rule.EvaluationFailedException;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
index 249d335..93326be 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterClientService.java
@@ -44,15 +44,14 @@ import org.apache.tajo.ipc.ClientProtos.*;
import org.apache.tajo.ipc.TajoMasterClientProtocol;
import org.apache.tajo.ipc.TajoMasterClientProtocol.TajoMasterClientProtocolService;
import org.apache.tajo.master.TajoMaster.MasterContext;
-import org.apache.tajo.master.querymaster.QueryInProgress;
-import org.apache.tajo.master.querymaster.QueryInfo;
-import org.apache.tajo.master.querymaster.QueryJobEvent;
-import org.apache.tajo.master.querymaster.QueryJobManager;
+import org.apache.tajo.master.exec.NonForwardQueryResultScanner;
+import org.apache.tajo.querymaster.QueryInProgress;
+import org.apache.tajo.querymaster.QueryJobEvent;
import org.apache.tajo.master.rm.Worker;
import org.apache.tajo.master.rm.WorkerResource;
-import org.apache.tajo.master.session.InvalidSessionException;
-import org.apache.tajo.master.session.NoSuchSessionVariableException;
-import org.apache.tajo.master.session.Session;
+import org.apache.tajo.session.InvalidSessionException;
+import org.apache.tajo.session.NoSuchSessionVariableException;
+import org.apache.tajo.session.Session;
import org.apache.tajo.rpc.BlockingRpcServer;
import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.BoolProto;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterService.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterService.java
index 1e3501c..a7df206 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterService.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMasterService.java
@@ -23,14 +23,12 @@ import com.google.protobuf.RpcController;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.proto.YarnProtos;
import org.apache.tajo.QueryId;
import org.apache.tajo.TajoIdProtos;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.ipc.ContainerProtocol;
import org.apache.tajo.ipc.TajoMasterProtocol;
import org.apache.tajo.master.cluster.WorkerConnectionInfo;
-import org.apache.tajo.master.querymaster.QueryJobManager;
import org.apache.tajo.master.rm.Worker;
import org.apache.tajo.master.rm.WorkerResource;
import org.apache.tajo.rpc.AsyncRpcServer;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerContext.java b/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerContext.java
deleted file mode 100644
index 755df5a..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerContext.java
+++ /dev/null
@@ -1,65 +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.tajo.master;
-
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.master.querymaster.QueryMasterTask;
-
-public class TaskSchedulerContext {
- private QueryMasterTask.QueryMasterTaskContext masterContext;
- private boolean isLeafQuery;
- private ExecutionBlockId blockId;
- private int taskSize;
- private int estimatedTaskNum;
-
- public TaskSchedulerContext(QueryMasterTask.QueryMasterTaskContext masterContext, boolean isLeafQuery,
- ExecutionBlockId blockId) {
- this.masterContext = masterContext;
- this.isLeafQuery = isLeafQuery;
- this.blockId = blockId;
- }
-
- public QueryMasterTask.QueryMasterTaskContext getMasterContext() {
- return masterContext;
- }
-
- public boolean isLeafQuery() {
- return isLeafQuery;
- }
-
- public ExecutionBlockId getBlockId() {
- return blockId;
- }
-
- public int getTaskSize() {
- return taskSize;
- }
-
- public int getEstimatedTaskNum() {
- return estimatedTaskNum;
- }
-
- public void setTaskSize(int taskSize) {
- this.taskSize = taskSize;
- }
-
- public void setEstimatedTaskNum(int estimatedTaskNum) {
- this.estimatedTaskNum = estimatedTaskNum;
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerFactory.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerFactory.java b/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerFactory.java
deleted file mode 100644
index e5291e9..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/TaskSchedulerFactory.java
+++ /dev/null
@@ -1,69 +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.tajo.master;
-
-import com.google.common.collect.Maps;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tajo.master.querymaster.Stage;
-
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.util.Map;
-
-public class TaskSchedulerFactory {
- private static Class<? extends AbstractTaskScheduler> CACHED_ALGORITHM_CLASS;
- private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE = Maps.newConcurrentMap();
- private static final Class<?>[] DEFAULT_PARAMS = { TaskSchedulerContext.class, Stage.class };
-
- public static Class<? extends AbstractTaskScheduler> getTaskSchedulerClass(Configuration conf)
- throws IOException {
- if (CACHED_ALGORITHM_CLASS != null) {
- return CACHED_ALGORITHM_CLASS;
- } else {
- CACHED_ALGORITHM_CLASS = conf.getClass("tajo.querymaster.task-scheduler", null, AbstractTaskScheduler.class);
- }
-
- if (CACHED_ALGORITHM_CLASS == null) {
- throw new IOException("Task scheduler is null");
- }
- return CACHED_ALGORITHM_CLASS;
- }
-
- public static <T extends AbstractTaskScheduler> T get(Class<T> clazz, TaskSchedulerContext context,
- Stage stage) {
- T result;
- try {
- Constructor<T> constructor = (Constructor<T>) CONSTRUCTOR_CACHE.get(clazz);
- if (constructor == null) {
- constructor = clazz.getDeclaredConstructor(DEFAULT_PARAMS);
- constructor.setAccessible(true);
- CONSTRUCTOR_CACHE.put(clazz, constructor);
- }
- result = constructor.newInstance(new Object[]{context, stage});
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- return result;
- }
-
- public static AbstractTaskScheduler get(Configuration conf, TaskSchedulerContext context, Stage stage)
- throws IOException {
- return get(getTaskSchedulerClass(conf), context, stage);
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/event/QueryCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryCompletedEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryCompletedEvent.java
index e5a9a32..fcae53c 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryCompletedEvent.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryCompletedEvent.java
@@ -19,7 +19,7 @@
package org.apache.tajo.master.event;
import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.master.querymaster.StageState;
+import org.apache.tajo.querymaster.StageState;
public class QueryCompletedEvent extends QueryEvent {
private final ExecutionBlockId executionBlockId;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
index 623576a..3a387fa 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
@@ -21,7 +21,7 @@ package org.apache.tajo.master.event;
import org.apache.hadoop.yarn.event.AbstractEvent;
import org.apache.tajo.QueryId;
import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.master.session.Session;
+import org.apache.tajo.session.Session;
/**
* This event is conveyed to QueryMaster.
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/event/StageCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/StageCompletedEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/StageCompletedEvent.java
index 2d16fbe..f012e92 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/StageCompletedEvent.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/StageCompletedEvent.java
@@ -19,7 +19,7 @@
package org.apache.tajo.master.event;
import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.master.querymaster.StageState;
+import org.apache.tajo.querymaster.StageState;
public class StageCompletedEvent extends QueryEvent {
private final ExecutionBlockId executionBlockId;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java
index 91ef942..5a016fb 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/event/TaskAttemptToSchedulerEvent.java
@@ -21,7 +21,7 @@ package org.apache.tajo.master.event;
import com.google.protobuf.RpcCallback;
import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.ipc.TajoWorkerProtocol;
-import org.apache.tajo.master.querymaster.TaskAttempt;
+import org.apache.tajo.querymaster.TaskAttempt;
import org.apache.tajo.master.container.TajoContainerId;
public class TaskAttemptToSchedulerEvent extends TaskSchedulerEvent {
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
index acbaa01..2030b55 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java
@@ -34,7 +34,6 @@ import org.apache.tajo.catalog.proto.CatalogProtos.AlterTablespaceProto;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.engine.query.QueryContext;
import org.apache.tajo.master.TajoMaster;
-import org.apache.tajo.master.session.Session;
import org.apache.tajo.plan.LogicalPlan;
import org.apache.tajo.plan.logical.*;
import org.apache.tajo.plan.util.PlannerUtil;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultFileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultFileScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultFileScanner.java
new file mode 100644
index 0000000..dc0c44a
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultFileScanner.java
@@ -0,0 +1,164 @@
+/**
+ * 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.tajo.master.exec;
+
+import com.google.protobuf.ByteString;
+
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.TaskAttemptId;
+import org.apache.tajo.TaskId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableDesc;
+import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.plan.logical.ScanNode;
+import org.apache.tajo.engine.planner.physical.SeqScanExec;
+import org.apache.tajo.engine.query.QueryContext;
+import org.apache.tajo.storage.RowStoreUtil;
+import org.apache.tajo.storage.RowStoreUtil.RowStoreEncoder;
+import org.apache.tajo.storage.StorageManager;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.storage.fragment.Fragment;
+import org.apache.tajo.storage.fragment.FragmentConvertor;
+import org.apache.tajo.worker.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class NonForwardQueryResultFileScanner implements NonForwardQueryResultScanner {
+ private static final int MAX_FRAGMENT_NUM_PER_SCAN = 100;
+
+ private QueryId queryId;
+ private String sessionId;
+ private SeqScanExec scanExec;
+ private TableDesc tableDesc;
+ private RowStoreEncoder rowEncoder;
+ private int maxRow;
+ private int currentNumRows;
+ private TaskAttemptContext taskContext;
+ private TajoConf tajoConf;
+ private ScanNode scanNode;
+
+ private int currentFragmentIndex = 0;
+
+ public NonForwardQueryResultFileScanner(TajoConf tajoConf, String sessionId, QueryId queryId, ScanNode scanNode,
+ TableDesc tableDesc, int maxRow) throws IOException {
+ this.tajoConf = tajoConf;
+ this.sessionId = sessionId;
+ this.queryId = queryId;
+ this.scanNode = scanNode;
+ this.tableDesc = tableDesc;
+ this.maxRow = maxRow;
+ this.rowEncoder = RowStoreUtil.createEncoder(tableDesc.getLogicalSchema());
+ }
+
+ public void init() throws IOException {
+ initSeqScanExec();
+ }
+
+ private void initSeqScanExec() throws IOException {
+ List<Fragment> fragments = StorageManager.getStorageManager(tajoConf, tableDesc.getMeta().getStoreType())
+ .getNonForwardSplit(tableDesc, currentFragmentIndex, MAX_FRAGMENT_NUM_PER_SCAN);
+
+ if (fragments != null && !fragments.isEmpty()) {
+ FragmentProto[] fragmentProtos = FragmentConvertor.toFragmentProtoArray(fragments.toArray(new Fragment[] {}));
+ this.taskContext = new TaskAttemptContext(
+ new QueryContext(tajoConf), null,
+ new TaskAttemptId(new TaskId(new ExecutionBlockId(queryId, 1), 0), 0),
+ fragmentProtos, null);
+ try {
+ // scanNode must be clone cause SeqScanExec change target in the case of
+ // a partitioned table.
+ scanExec = new SeqScanExec(taskContext, (ScanNode) scanNode.clone(), fragmentProtos);
+ } catch (CloneNotSupportedException e) {
+ throw new IOException(e.getMessage(), e);
+ }
+ scanExec.init();
+ currentFragmentIndex += fragments.size();
+ }
+ }
+
+ public QueryId getQueryId() {
+ return queryId;
+ }
+
+ public String getSessionId() {
+ return sessionId;
+ }
+
+ public void setScanExec(SeqScanExec scanExec) {
+ this.scanExec = scanExec;
+ }
+
+ public TableDesc getTableDesc() {
+ return tableDesc;
+ }
+
+ public void close() throws Exception {
+ if (scanExec != null) {
+ scanExec.close();
+ scanExec = null;
+ }
+ }
+
+ public List<ByteString> getNextRows(int fetchRowNum) throws IOException {
+ List<ByteString> rows = new ArrayList<ByteString>();
+ if (scanExec == null) {
+ return rows;
+ }
+ int rowCount = 0;
+ while (true) {
+ Tuple tuple = scanExec.next();
+ if (tuple == null) {
+ scanExec.close();
+ scanExec = null;
+ initSeqScanExec();
+ if (scanExec != null) {
+ tuple = scanExec.next();
+ }
+ if (tuple == null) {
+ if (scanExec != null) {
+ scanExec.close();
+ scanExec = null;
+ }
+ break;
+ }
+ }
+ rows.add(ByteString.copyFrom((rowEncoder.toBytes(tuple))));
+ rowCount++;
+ currentNumRows++;
+ if (rowCount >= fetchRowNum) {
+ break;
+ }
+ if (currentNumRows >= maxRow) {
+ scanExec.close();
+ scanExec = null;
+ break;
+ }
+ }
+ return rows;
+ }
+
+ @Override
+ public Schema getLogicalSchema() {
+ return tableDesc.getLogicalSchema();
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultScanner.java
new file mode 100644
index 0000000..86d2843
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/NonForwardQueryResultScanner.java
@@ -0,0 +1,46 @@
+/**
+ * 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.tajo.master.exec;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.tajo.QueryId;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.catalog.TableDesc;
+
+import com.google.protobuf.ByteString;
+
+public interface NonForwardQueryResultScanner {
+
+ public void close() throws Exception;
+
+ public Schema getLogicalSchema();
+
+ public List<ByteString> getNextRows(int fetchRowNum) throws IOException;
+
+ public QueryId getQueryId();
+
+ public String getSessionId();
+
+ public TableDesc getTableDesc();
+
+ public void init() throws IOException;
+
+}
[14/16] tajo git commit: TAJO-1288: Refactoring
org.apache.tajo.master package.
Posted by ji...@apache.org.
TAJO-1288: Refactoring org.apache.tajo.master package.
Closes #338
Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/1c29c1cb
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/1c29c1cb
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/1c29c1cb
Branch: refs/heads/index_support
Commit: 1c29c1cb4bd0e2d75954575717cb5cf05875fe51
Parents: a1e0328
Author: Hyunsik Choi <hy...@apache.org>
Authored: Fri Jan 9 00:31:54 2015 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Fri Jan 9 00:31:54 2015 +0900
----------------------------------------------------------------------
CHANGES | 2 +
.../apache/tajo/engine/query/QueryContext.java | 2 +-
.../main/java/org/apache/tajo/ha/HAService.java | 56 +
.../org/apache/tajo/ha/HAServiceHDFSImpl.java | 316 +++++
.../java/org/apache/tajo/ha/TajoMasterInfo.java | 89 ++
.../tajo/master/AbstractTaskScheduler.java | 56 -
.../org/apache/tajo/master/ContainerProxy.java | 2 +-
.../tajo/master/DefaultTaskScheduler.java | 928 ------------
.../apache/tajo/master/FetchScheduleEvent.java | 40 -
.../org/apache/tajo/master/FragmentPair.java | 73 -
.../org/apache/tajo/master/GlobalEngine.java | 2 +-
.../NonForwardQueryResultFileScanner.java | 164 ---
.../master/NonForwardQueryResultScanner.java | 46 -
.../NonForwardQueryResultSystemScanner.java | 616 --------
.../java/org/apache/tajo/master/QueryInfo.java | 235 +++
.../org/apache/tajo/master/QueryJobManager.java | 311 ++++
.../apache/tajo/master/ScheduledFetches.java | 49 -
.../apache/tajo/master/TajoContainerProxy.java | 2 +-
.../java/org/apache/tajo/master/TajoMaster.java | 11 +-
.../tajo/master/TajoMasterClientService.java | 13 +-
.../apache/tajo/master/TajoMasterService.java | 2 -
.../tajo/master/TaskSchedulerContext.java | 65 -
.../tajo/master/TaskSchedulerFactory.java | 69 -
.../tajo/master/event/QueryCompletedEvent.java | 2 +-
.../tajo/master/event/QueryStartEvent.java | 2 +-
.../tajo/master/event/StageCompletedEvent.java | 2 +-
.../event/TaskAttemptToSchedulerEvent.java | 2 +-
.../apache/tajo/master/exec/DDLExecutor.java | 1 -
.../exec/NonForwardQueryResultFileScanner.java | 164 +++
.../exec/NonForwardQueryResultScanner.java | 46 +
.../NonForwardQueryResultSystemScanner.java | 616 ++++++++
.../apache/tajo/master/exec/QueryExecutor.java | 9 +-
.../org/apache/tajo/master/ha/HAService.java | 56 -
.../tajo/master/ha/HAServiceHDFSImpl.java | 318 -----
.../apache/tajo/master/ha/TajoMasterInfo.java | 89 --
.../master/metrics/CatalogMetricsGaugeSet.java | 56 -
.../metrics/WorkerResourceMetricsGaugeSet.java | 74 -
.../apache/tajo/master/querymaster/Query.java | 738 ----------
.../master/querymaster/QueryInProgress.java | 300 ----
.../tajo/master/querymaster/QueryInfo.java | 235 ---
.../tajo/master/querymaster/QueryJobEvent.java | 45 -
.../master/querymaster/QueryJobManager.java | 310 ----
.../tajo/master/querymaster/QueryMaster.java | 631 --------
.../querymaster/QueryMasterManagerService.java | 263 ----
.../master/querymaster/QueryMasterRunner.java | 149 --
.../master/querymaster/QueryMasterTask.java | 638 ---------
.../tajo/master/querymaster/Repartitioner.java | 1251 ----------------
.../apache/tajo/master/querymaster/Stage.java | 1342 ------------------
.../tajo/master/querymaster/StageState.java | 30 -
.../apache/tajo/master/querymaster/Task.java | 907 ------------
.../tajo/master/querymaster/TaskAttempt.java | 443 ------
.../master/rm/TajoWorkerResourceManager.java | 3 +-
.../tajo/master/rm/WorkerResourceManager.java | 2 +-
.../master/scheduler/QuerySchedulingInfo.java | 55 +
.../apache/tajo/master/scheduler/Scheduler.java | 41 +
.../master/scheduler/SchedulingAlgorithms.java | 47 +
.../master/scheduler/SimpleFifoScheduler.java | 147 ++
.../master/session/InvalidSessionException.java | 25 -
.../session/NoSuchSessionVariableException.java | 25 -
.../org/apache/tajo/master/session/Session.java | 196 ---
.../tajo/master/session/SessionConstants.java | 23 -
.../tajo/master/session/SessionEvent.java | 34 -
.../tajo/master/session/SessionEventType.java | 24 -
.../session/SessionLivelinessMonitor.java | 53 -
.../tajo/master/session/SessionManager.java | 144 --
.../tajo/metrics/CatalogMetricsGaugeSet.java | 56 +
.../metrics/WorkerResourceMetricsGaugeSet.java | 74 +
.../tajo/querymaster/AbstractTaskScheduler.java | 56 +
.../tajo/querymaster/DefaultTaskScheduler.java | 926 ++++++++++++
.../tajo/querymaster/FetchScheduleEvent.java | 40 +
.../java/org/apache/tajo/querymaster/Query.java | 738 ++++++++++
.../tajo/querymaster/QueryInProgress.java | 301 ++++
.../apache/tajo/querymaster/QueryJobEvent.java | 46 +
.../apache/tajo/querymaster/QueryMaster.java | 631 ++++++++
.../querymaster/QueryMasterManagerService.java | 262 ++++
.../tajo/querymaster/QueryMasterTask.java | 638 +++++++++
.../apache/tajo/querymaster/Repartitioner.java | 1250 ++++++++++++++++
.../java/org/apache/tajo/querymaster/Stage.java | 1342 ++++++++++++++++++
.../org/apache/tajo/querymaster/StageState.java | 30 +
.../java/org/apache/tajo/querymaster/Task.java | 897 ++++++++++++
.../apache/tajo/querymaster/TaskAttempt.java | 443 ++++++
.../tajo/querymaster/TaskSchedulerContext.java | 65 +
.../tajo/querymaster/TaskSchedulerFactory.java | 68 +
.../tajo/scheduler/QuerySchedulingInfo.java | 55 -
.../org/apache/tajo/scheduler/Scheduler.java | 41 -
.../tajo/scheduler/SchedulingAlgorithms.java | 47 -
.../tajo/scheduler/SimpleFifoScheduler.java | 147 --
.../tajo/session/InvalidSessionException.java | 25 +
.../session/NoSuchSessionVariableException.java | 25 +
.../java/org/apache/tajo/session/Session.java | 196 +++
.../apache/tajo/session/SessionConstants.java | 23 +
.../org/apache/tajo/session/SessionEvent.java | 34 +
.../apache/tajo/session/SessionEventType.java | 24 +
.../tajo/session/SessionLivelinessMonitor.java | 53 +
.../org/apache/tajo/session/SessionManager.java | 144 ++
.../main/java/org/apache/tajo/util/JSPUtil.java | 10 +-
.../apache/tajo/util/history/HistoryReader.java | 2 +-
.../apache/tajo/util/history/HistoryWriter.java | 2 +-
.../java/org/apache/tajo/worker/FetchImpl.java | 4 +-
.../tajo/worker/TajoResourceAllocator.java | 6 +-
.../java/org/apache/tajo/worker/TajoWorker.java | 6 +-
.../tajo/worker/TajoWorkerClientService.java | 2 +-
tajo-core/src/main/resources/tajo-default.xml | 2 +-
.../resources/webapps/admin/catalogview.jsp | 2 +-
.../main/resources/webapps/admin/cluster.jsp | 4 +-
.../src/main/resources/webapps/admin/index.jsp | 6 +-
.../src/main/resources/webapps/admin/query.jsp | 4 +-
.../resources/webapps/admin/query_executor.jsp | 2 +-
.../src/main/resources/webapps/worker/index.jsp | 4 +-
.../resources/webapps/worker/querydetail.jsp | 4 +-
.../main/resources/webapps/worker/queryplan.jsp | 6 +-
.../resources/webapps/worker/querytasks.jsp | 2 +-
.../src/main/resources/webapps/worker/task.jsp | 8 +-
.../apache/tajo/LocalTajoTestingUtility.java | 2 +-
.../org/apache/tajo/TajoTestingCluster.java | 8 +-
.../tajo/engine/planner/TestLogicalPlanner.java | 2 +-
.../planner/physical/TestPhysicalPlanner.java | 2 +-
.../tajo/engine/query/TestGroupByQuery.java | 8 +-
.../tajo/engine/query/TestJoinBroadcast.java | 2 +-
.../tajo/engine/query/TestTablePartitions.java | 2 +-
.../apache/tajo/ha/TestHAServiceHDFSImpl.java | 153 ++
.../TestNonForwardQueryResultSystemScanner.java | 4 +-
.../apache/tajo/master/TestRepartitioner.java | 8 +-
.../tajo/master/ha/TestHAServiceHDFSImpl.java | 158 ---
.../querymaster/TestIntermediateEntry.java | 53 -
.../tajo/master/querymaster/TestKillQuery.java | 125 --
.../master/querymaster/TestQueryProgress.java | 75 -
.../querymaster/TestTaskStatusUpdate.java | 194 ---
.../master/scheduler/TestFifoScheduler.java | 116 ++
.../tajo/querymaster/TestIntermediateEntry.java | 53 +
.../apache/tajo/querymaster/TestKillQuery.java | 125 ++
.../tajo/querymaster/TestQueryProgress.java | 75 +
.../tajo/querymaster/TestTaskStatusUpdate.java | 194 +++
.../tajo/scheduler/TestFifoScheduler.java | 116 --
.../java/org/apache/tajo/util/TestJSPUtil.java | 2 +-
.../util/history/TestHistoryWriterReader.java | 2 +-
.../org/apache/tajo/worker/TestHistory.java | 2 +-
tajo-dist/pom.xml | 8 +-
138 files changed, 11317 insertions(+), 11612 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 7783db8..96b63ea 100644
--- a/CHANGES
+++ b/CHANGES
@@ -27,6 +27,8 @@ Release 0.9.1 - unreleased
IMPROVEMENT
+ TAJO-1288: Refactoring org.apache.tajo.master package. (hyunsik)
+
TAJO-1279: Cleanup TajoAsyncDispatcher and interrupt stop events.
(jinho)
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
index 493ca6e..7b3c00d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java
@@ -25,7 +25,7 @@ import org.apache.tajo.QueryVars;
import org.apache.tajo.SessionVars;
import org.apache.tajo.catalog.partition.PartitionMethodDesc;
import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.master.session.Session;
+import org.apache.tajo.session.Session;
import org.apache.tajo.plan.logical.NodeType;
import static org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueSetProto;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/ha/HAService.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/ha/HAService.java b/tajo-core/src/main/java/org/apache/tajo/ha/HAService.java
new file mode 100644
index 0000000..1329223
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/ha/HAService.java
@@ -0,0 +1,56 @@
+/**
+ * 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.tajo.ha;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * The HAService is responsible for setting active TajoMaster on startup or when the
+ * current active is changing (eg due to failure), monitoring the health of TajoMaster.
+ *
+ */
+public interface HAService {
+
+ /**
+ * Add master name to shared storage.
+ */
+ public void register() throws IOException;
+
+
+ /**
+ * Delete master name to shared storage.
+ *
+ */
+ public void delete() throws IOException;
+
+ /**
+ *
+ * @return True if current master is an active master.
+ */
+ public boolean isActiveStatus();
+
+ /**
+ *
+ * @return return all master list
+ * @throws IOException
+ */
+ public List<TajoMasterInfo> getMasters() throws IOException;
+
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/ha/HAServiceHDFSImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/ha/HAServiceHDFSImpl.java b/tajo-core/src/main/java/org/apache/tajo/ha/HAServiceHDFSImpl.java
new file mode 100644
index 0000000..e18a9b2
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/ha/HAServiceHDFSImpl.java
@@ -0,0 +1,316 @@
+/**
+ * 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.tajo.ha;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.tajo.TajoConstants;
+import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.master.TajoMaster;
+import org.apache.tajo.master.TajoMaster.MasterContext;
+import org.apache.tajo.util.TUtil;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.List;
+
+/**
+ * This implements HAService utilizing HDFS cluster. This saves master status to HDFS cluster.
+ *
+ */
+public class HAServiceHDFSImpl implements HAService {
+ private static Log LOG = LogFactory.getLog(HAServiceHDFSImpl.class);
+
+ private MasterContext context;
+ private TajoConf conf;
+
+ private FileSystem fs;
+
+ private String masterName;
+ private Path rootPath;
+ private Path haPath;
+ private Path activePath;
+ private Path backupPath;
+
+ private boolean isActiveStatus = false;
+
+ //thread which runs periodically to see the last time since a heartbeat is received.
+ private Thread checkerThread;
+ private volatile boolean stopped = false;
+
+ private int monitorInterval;
+
+ private String currentActiveMaster;
+
+ public HAServiceHDFSImpl(MasterContext context) throws IOException {
+ this.context = context;
+ this.conf = context.getConf();
+ initSystemDirectory();
+
+ InetSocketAddress socketAddress = context.getTajoMasterService().getBindAddress();
+ this.masterName = socketAddress.getAddress().getHostAddress() + ":" + socketAddress.getPort();
+
+ monitorInterval = conf.getIntVar(TajoConf.ConfVars.TAJO_MASTER_HA_MONITOR_INTERVAL);
+ }
+
+ private void initSystemDirectory() throws IOException {
+ // Get Tajo root dir
+ this.rootPath = TajoConf.getTajoRootDir(conf);
+
+ // Check Tajo root dir
+ this.fs = rootPath.getFileSystem(conf);
+
+ // Check and create Tajo system HA dir
+ haPath = TajoConf.getSystemHADir(conf);
+ if (!fs.exists(haPath)) {
+ fs.mkdirs(haPath, new FsPermission(TajoMaster.SYSTEM_RESOURCE_DIR_PERMISSION));
+ LOG.info("System HA dir '" + haPath + "' is created");
+ }
+
+ activePath = new Path(haPath, TajoConstants.SYSTEM_HA_ACTIVE_DIR_NAME);
+ if (!fs.exists(activePath)) {
+ fs.mkdirs(activePath, new FsPermission(TajoMaster.SYSTEM_RESOURCE_DIR_PERMISSION));
+ LOG.info("System HA Active dir '" + activePath + "' is created");
+ }
+
+ backupPath = new Path(haPath, TajoConstants.SYSTEM_HA_BACKUP_DIR_NAME);
+ if (!fs.exists(backupPath)) {
+ fs.mkdirs(backupPath, new FsPermission(TajoMaster.SYSTEM_RESOURCE_DIR_PERMISSION));
+ LOG.info("System HA Backup dir '" + backupPath + "' is created");
+ }
+ }
+
+ private void startPingChecker() {
+ if (checkerThread == null) {
+ checkerThread = new Thread(new PingChecker());
+ checkerThread.setName("Ping Checker");
+ checkerThread.start();
+ }
+ }
+
+ @Override
+ public void register() throws IOException {
+ FileStatus[] files = fs.listStatus(activePath);
+
+ // Phase 1: If there is not another active master, this try to become active master.
+ if (files.length == 0) {
+ createMasterFile(true);
+ currentActiveMaster = masterName;
+ LOG.info(String.format("This is added to active master (%s)", masterName));
+ } else {
+ // Phase 2: If there is active master information, we need to check its status.
+ Path activePath = files[0].getPath();
+ currentActiveMaster = activePath.getName().replaceAll("_", ":");
+
+ // Phase 3: If current active master is dead, this master should be active master.
+ if (!HAServiceUtil.isMasterAlive(currentActiveMaster, conf)) {
+ fs.delete(activePath, true);
+ createMasterFile(true);
+ currentActiveMaster = masterName;
+ LOG.info(String.format("This is added to active master (%s)", masterName));
+ } else {
+ // Phase 4: If current active master is alive, this master need to be backup master.
+ createMasterFile(false);
+ LOG.info(String.format("This is added to backup masters (%s)", masterName));
+ }
+ }
+ }
+
+ private void createMasterFile(boolean isActive) throws IOException {
+ String fileName = masterName.replaceAll(":", "_");
+ Path path = null;
+
+ if (isActive) {
+ path = new Path(activePath, fileName);
+ } else {
+ path = new Path(backupPath, fileName);
+ }
+
+ StringBuilder sb = new StringBuilder();
+ InetSocketAddress address = getHostAddress(HAConstants.MASTER_CLIENT_RPC_ADDRESS);
+ sb.append(address.getAddress().getHostAddress()).append(":").append(address.getPort()).append("_");
+
+ address = getHostAddress(HAConstants.RESOURCE_TRACKER_RPC_ADDRESS);
+ sb.append(address.getAddress().getHostAddress()).append(":").append(address.getPort()).append("_");
+
+ address = getHostAddress(HAConstants.CATALOG_ADDRESS);
+ sb.append(address.getAddress().getHostAddress()).append(":").append(address.getPort()).append("_");
+
+ address = getHostAddress(HAConstants.MASTER_INFO_ADDRESS);
+ sb.append(address.getAddress().getHostAddress()).append(":").append(address.getPort());
+
+ FSDataOutputStream out = fs.create(path);
+
+ try {
+ out.writeUTF(sb.toString());
+ out.hflush();
+ out.close();
+ } catch (FileAlreadyExistsException e) {
+ createMasterFile(false);
+ }
+
+ if (isActive) {
+ isActiveStatus = true;
+ } else {
+ isActiveStatus = false;
+ }
+
+ startPingChecker();
+ }
+
+
+ private InetSocketAddress getHostAddress(int type) {
+ InetSocketAddress address = null;
+
+ switch (type) {
+ case HAConstants.MASTER_UMBILICAL_RPC_ADDRESS:
+ address = context.getConf().getSocketAddrVar(TajoConf.ConfVars
+ .TAJO_MASTER_UMBILICAL_RPC_ADDRESS);
+ break;
+ case HAConstants.MASTER_CLIENT_RPC_ADDRESS:
+ address = context.getConf().getSocketAddrVar(TajoConf.ConfVars
+ .TAJO_MASTER_CLIENT_RPC_ADDRESS);
+ break;
+ case HAConstants.RESOURCE_TRACKER_RPC_ADDRESS:
+ address = context.getConf().getSocketAddrVar(TajoConf.ConfVars
+ .RESOURCE_TRACKER_RPC_ADDRESS);
+ break;
+ case HAConstants.CATALOG_ADDRESS:
+ address = context.getConf().getSocketAddrVar(TajoConf.ConfVars
+ .CATALOG_ADDRESS);
+ break;
+ case HAConstants.MASTER_INFO_ADDRESS:
+ address = context.getConf().getSocketAddrVar(TajoConf.ConfVars
+ .TAJO_MASTER_INFO_ADDRESS);
+ default:
+ break;
+ }
+
+ return NetUtils.createSocketAddr(masterName.split(":")[0] + ":" + address.getPort());
+ }
+
+ @Override
+ public void delete() throws IOException {
+ String fileName = masterName.replaceAll(":", "_");
+
+ Path activeFile = new Path(activePath, fileName);
+ if (fs.exists(activeFile)) {
+ fs.delete(activeFile, true);
+ }
+
+ Path backupFile = new Path(backupPath, fileName);
+ if (fs.exists(backupFile)) {
+ fs.delete(backupFile, true);
+ }
+ if (isActiveStatus) {
+ isActiveStatus = false;
+ }
+ stopped = true;
+ }
+
+ @Override
+ public boolean isActiveStatus() {
+ return isActiveStatus;
+ }
+
+ @Override
+ public List<TajoMasterInfo> getMasters() throws IOException {
+ List<TajoMasterInfo> list = TUtil.newList();
+ Path path = null;
+
+ FileStatus[] files = fs.listStatus(activePath);
+ if (files.length == 1) {
+ path = files[0].getPath();
+ list.add(createTajoMasterInfo(path, true));
+ }
+
+ files = fs.listStatus(backupPath);
+ for (FileStatus status : files) {
+ path = status.getPath();
+ list.add(createTajoMasterInfo(path, false));
+ }
+
+ return list;
+ }
+
+ private TajoMasterInfo createTajoMasterInfo(Path path, boolean isActive) throws IOException {
+ String masterAddress = path.getName().replaceAll("_", ":");
+ boolean isAlive = HAServiceUtil.isMasterAlive(masterAddress, conf);
+
+ FSDataInputStream stream = fs.open(path);
+ String data = stream.readUTF();
+
+ stream.close();
+
+ String[] addresses = data.split("_");
+ TajoMasterInfo info = new TajoMasterInfo();
+
+ info.setTajoMasterAddress(NetUtils.createSocketAddr(masterAddress));
+ info.setTajoClientAddress(NetUtils.createSocketAddr(addresses[0]));
+ info.setWorkerResourceTrackerAddr(NetUtils.createSocketAddr(addresses[1]));
+ info.setCatalogAddress(NetUtils.createSocketAddr(addresses[2]));
+ info.setWebServerAddress(NetUtils.createSocketAddr(addresses[3]));
+
+ info.setAvailable(isAlive);
+ info.setActive(isActive);
+
+ return info;
+ }
+
+ private class PingChecker implements Runnable {
+ @Override
+ public void run() {
+ while (!stopped && !Thread.currentThread().isInterrupted()) {
+ synchronized (HAServiceHDFSImpl.this) {
+ try {
+ if (!currentActiveMaster.equals(masterName)) {
+ boolean isAlive = HAServiceUtil.isMasterAlive(currentActiveMaster, conf);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("currentActiveMaster:" + currentActiveMaster + ", thisMasterName:" + masterName
+ + ", isAlive:" + isAlive);
+ }
+
+ // If active master is dead, this master should be active master instead of
+ // previous active master.
+ if (!isAlive) {
+ FileStatus[] files = fs.listStatus(activePath);
+ if (files.length == 0 || (files.length == 1
+ && currentActiveMaster.equals(files[0].getPath().getName().replaceAll("_", ":")))) {
+ delete();
+ register();
+ }
+ }
+ }
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ try {
+ Thread.sleep(monitorInterval);
+ } catch (InterruptedException e) {
+ LOG.info("PingChecker interrupted. - masterName:" + masterName);
+ break;
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/ha/TajoMasterInfo.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/ha/TajoMasterInfo.java b/tajo-core/src/main/java/org/apache/tajo/ha/TajoMasterInfo.java
new file mode 100644
index 0000000..c6fdd40
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/ha/TajoMasterInfo.java
@@ -0,0 +1,89 @@
+/**
+ * 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.tajo.ha;
+
+import java.net.InetSocketAddress;
+
+public class TajoMasterInfo {
+
+ private boolean available;
+ private boolean isActive;
+
+ private InetSocketAddress tajoMasterAddress;
+ private InetSocketAddress tajoClientAddress;
+ private InetSocketAddress workerResourceTrackerAddr;
+ private InetSocketAddress catalogAddress;
+ private InetSocketAddress webServerAddress;
+
+ public InetSocketAddress getTajoMasterAddress() {
+ return tajoMasterAddress;
+ }
+
+ public void setTajoMasterAddress(InetSocketAddress tajoMasterAddress) {
+ this.tajoMasterAddress = tajoMasterAddress;
+ }
+
+ public InetSocketAddress getTajoClientAddress() {
+ return tajoClientAddress;
+ }
+
+ public void setTajoClientAddress(InetSocketAddress tajoClientAddress) {
+ this.tajoClientAddress = tajoClientAddress;
+ }
+
+ public InetSocketAddress getWorkerResourceTrackerAddr() {
+ return workerResourceTrackerAddr;
+ }
+
+ public void setWorkerResourceTrackerAddr(InetSocketAddress workerResourceTrackerAddr) {
+ this.workerResourceTrackerAddr = workerResourceTrackerAddr;
+ }
+
+ public InetSocketAddress getCatalogAddress() {
+ return catalogAddress;
+ }
+
+ public void setCatalogAddress(InetSocketAddress catalogAddress) {
+ this.catalogAddress = catalogAddress;
+ }
+
+ public InetSocketAddress getWebServerAddress() {
+ return webServerAddress;
+ }
+
+ public void setWebServerAddress(InetSocketAddress webServerAddress) {
+ this.webServerAddress = webServerAddress;
+ }
+
+ public boolean isAvailable() {
+ return available;
+ }
+
+ public void setAvailable(boolean available) {
+ this.available = available;
+ }
+
+ public boolean isActive() {
+ return isActive;
+ }
+
+ public void setActive(boolean active) {
+ isActive = active;
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/AbstractTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/AbstractTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/AbstractTaskScheduler.java
deleted file mode 100644
index 320a5aa..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/AbstractTaskScheduler.java
+++ /dev/null
@@ -1,56 +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.tajo.master;
-
-import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.tajo.master.event.TaskRequestEvent;
-import org.apache.tajo.master.event.TaskSchedulerEvent;
-
-
-public abstract class AbstractTaskScheduler extends AbstractService implements EventHandler<TaskSchedulerEvent> {
-
- protected int hostLocalAssigned;
- protected int rackLocalAssigned;
- protected int totalAssigned;
-
- /**
- * Construct the service.
- *
- * @param name service name
- */
- public AbstractTaskScheduler(String name) {
- super(name);
- }
-
- public int getHostLocalAssigned() {
- return hostLocalAssigned;
- }
-
- public int getRackLocalAssigned() {
- return rackLocalAssigned;
- }
-
- public int getTotalAssigned() {
- return totalAssigned;
- }
-
- public abstract void handleTaskRequestEvent(TaskRequestEvent event);
- public abstract int remainingScheduledObjectNum();
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java b/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java
index 462de91..562790d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/ContainerProxy.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.master.querymaster.QueryMasterTask;
+import org.apache.tajo.querymaster.QueryMasterTask;
import org.apache.tajo.master.container.TajoContainer;
import org.apache.tajo.master.container.TajoContainerId;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java b/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
deleted file mode 100644
index d47c93a..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/DefaultTaskScheduler.java
+++ /dev/null
@@ -1,928 +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.tajo.master;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.util.RackResolver;
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.QueryIdFactory;
-import org.apache.tajo.TaskAttemptId;
-import org.apache.tajo.engine.planner.global.ExecutionBlock;
-import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.engine.query.TaskRequest;
-import org.apache.tajo.engine.query.TaskRequestImpl;
-import org.apache.tajo.ipc.TajoWorkerProtocol;
-import org.apache.tajo.master.cluster.WorkerConnectionInfo;
-import org.apache.tajo.master.container.TajoContainerId;
-import org.apache.tajo.master.event.*;
-import org.apache.tajo.master.event.TaskAttemptToSchedulerEvent.TaskAttemptScheduleContext;
-import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
-import org.apache.tajo.master.querymaster.Stage;
-import org.apache.tajo.master.querymaster.Task;
-import org.apache.tajo.master.querymaster.TaskAttempt;
-import org.apache.tajo.plan.serder.LogicalNodeSerializer;
-import org.apache.tajo.plan.serder.PlanProto;
-import org.apache.tajo.storage.DataLocation;
-import org.apache.tajo.storage.fragment.FileFragment;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.util.NetUtils;
-import org.apache.tajo.worker.FetchImpl;
-
-import java.util.*;
-import java.util.Map.Entry;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
-
-public class DefaultTaskScheduler extends AbstractTaskScheduler {
- private static final Log LOG = LogFactory.getLog(DefaultTaskScheduler.class);
-
- private final TaskSchedulerContext context;
- private Stage stage;
-
- private Thread schedulingThread;
- private AtomicBoolean stopEventHandling = new AtomicBoolean(false);
-
- private ScheduledRequests scheduledRequests;
- private TaskRequests taskRequests;
-
- private int nextTaskId = 0;
- private int scheduledObjectNum = 0;
-
- public DefaultTaskScheduler(TaskSchedulerContext context, Stage stage) {
- super(DefaultTaskScheduler.class.getName());
- this.context = context;
- this.stage = stage;
- }
-
- @Override
- public void init(Configuration conf) {
-
- scheduledRequests = new ScheduledRequests();
- taskRequests = new TaskRequests();
-
- super.init(conf);
- }
-
- @Override
- public void start() {
- LOG.info("Start TaskScheduler");
-
- this.schedulingThread = new Thread() {
- public void run() {
-
- while(!stopEventHandling.get() && !Thread.currentThread().isInterrupted()) {
- try {
- synchronized (schedulingThread){
- schedulingThread.wait(100);
- }
- schedule();
- } catch (InterruptedException e) {
- break;
- } catch (Throwable e) {
- LOG.fatal(e.getMessage(), e);
- break;
- }
- }
- LOG.info("TaskScheduler schedulingThread stopped");
- }
- };
-
- this.schedulingThread.start();
- super.start();
- }
-
- private static final TaskAttemptId NULL_ATTEMPT_ID;
- public static final TajoWorkerProtocol.TaskRequestProto stopTaskRunnerReq;
- static {
- ExecutionBlockId nullStage = QueryIdFactory.newExecutionBlockId(QueryIdFactory.NULL_QUERY_ID, 0);
- NULL_ATTEMPT_ID = QueryIdFactory.newTaskAttemptId(QueryIdFactory.newTaskId(nullStage, 0), 0);
-
- TajoWorkerProtocol.TaskRequestProto.Builder builder =
- TajoWorkerProtocol.TaskRequestProto.newBuilder();
- builder.setId(NULL_ATTEMPT_ID.getProto());
- builder.setShouldDie(true);
- builder.setOutputTable("");
- builder.setPlan(PlanProto.LogicalNodeTree.newBuilder());
- builder.setClusteredOutput(false);
- stopTaskRunnerReq = builder.build();
- }
-
- @Override
- public void stop() {
- if(stopEventHandling.getAndSet(true)){
- return;
- }
-
- if (schedulingThread != null) {
- synchronized (schedulingThread) {
- schedulingThread.notifyAll();
- }
- }
-
- // Return all of request callbacks instantly.
- if(taskRequests != null){
- for (TaskRequestEvent req : taskRequests.taskRequestQueue) {
- req.getCallback().run(stopTaskRunnerReq);
- }
- }
-
- LOG.info("Task Scheduler stopped");
- super.stop();
- }
-
- private Fragment[] fragmentsForNonLeafTask;
- private Fragment[] broadcastFragmentsForNonLeafTask;
-
- LinkedList<TaskRequestEvent> taskRequestEvents = new LinkedList<TaskRequestEvent>();
- public void schedule() {
-
- if (taskRequests.size() > 0) {
- if (scheduledRequests.leafTaskNum() > 0) {
- LOG.debug("Try to schedule tasks with taskRequestEvents: " +
- taskRequests.size() + ", LeafTask Schedule Request: " +
- scheduledRequests.leafTaskNum());
- taskRequests.getTaskRequests(taskRequestEvents,
- scheduledRequests.leafTaskNum());
- LOG.debug("Get " + taskRequestEvents.size() + " taskRequestEvents ");
- if (taskRequestEvents.size() > 0) {
- scheduledRequests.assignToLeafTasks(taskRequestEvents);
- taskRequestEvents.clear();
- }
- }
- }
-
- if (taskRequests.size() > 0) {
- if (scheduledRequests.nonLeafTaskNum() > 0) {
- LOG.debug("Try to schedule tasks with taskRequestEvents: " +
- taskRequests.size() + ", NonLeafTask Schedule Request: " +
- scheduledRequests.nonLeafTaskNum());
- taskRequests.getTaskRequests(taskRequestEvents,
- scheduledRequests.nonLeafTaskNum());
- scheduledRequests.assignToNonLeafTasks(taskRequestEvents);
- taskRequestEvents.clear();
- }
- }
- }
-
- @Override
- public void handle(TaskSchedulerEvent event) {
- if (event.getType() == EventType.T_SCHEDULE) {
- if (event instanceof FragmentScheduleEvent) {
- FragmentScheduleEvent castEvent = (FragmentScheduleEvent) event;
- if (context.isLeafQuery()) {
- TaskAttemptScheduleContext taskContext = new TaskAttemptScheduleContext();
- Task task = Stage.newEmptyTask(context, taskContext, stage, nextTaskId++);
- task.addFragment(castEvent.getLeftFragment(), true);
- scheduledObjectNum++;
- if (castEvent.hasRightFragments()) {
- task.addFragments(castEvent.getRightFragments());
- }
- stage.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
- } else {
- fragmentsForNonLeafTask = new FileFragment[2];
- fragmentsForNonLeafTask[0] = castEvent.getLeftFragment();
- if (castEvent.hasRightFragments()) {
- FileFragment[] rightFragments = castEvent.getRightFragments().toArray(new FileFragment[]{});
- fragmentsForNonLeafTask[1] = rightFragments[0];
- if (rightFragments.length > 1) {
- broadcastFragmentsForNonLeafTask = new FileFragment[rightFragments.length - 1];
- System.arraycopy(rightFragments, 1, broadcastFragmentsForNonLeafTask, 0, broadcastFragmentsForNonLeafTask.length);
- } else {
- broadcastFragmentsForNonLeafTask = null;
- }
- }
- }
- } else if (event instanceof FetchScheduleEvent) {
- FetchScheduleEvent castEvent = (FetchScheduleEvent) event;
- Map<String, List<FetchImpl>> fetches = castEvent.getFetches();
- TaskAttemptScheduleContext taskScheduleContext = new TaskAttemptScheduleContext();
- Task task = Stage.newEmptyTask(context, taskScheduleContext, stage, nextTaskId++);
- scheduledObjectNum++;
- for (Entry<String, List<FetchImpl>> eachFetch : fetches.entrySet()) {
- task.addFetches(eachFetch.getKey(), eachFetch.getValue());
- task.addFragment(fragmentsForNonLeafTask[0], true);
- if (fragmentsForNonLeafTask[1] != null) {
- task.addFragment(fragmentsForNonLeafTask[1], true);
- }
- }
- if (broadcastFragmentsForNonLeafTask != null && broadcastFragmentsForNonLeafTask.length > 0) {
- task.addFragments(Arrays.asList(broadcastFragmentsForNonLeafTask));
- }
- stage.getEventHandler().handle(new TaskEvent(task.getId(), TaskEventType.T_SCHEDULE));
- } else if (event instanceof TaskAttemptToSchedulerEvent) {
- TaskAttemptToSchedulerEvent castEvent = (TaskAttemptToSchedulerEvent) event;
- if (context.isLeafQuery()) {
- scheduledRequests.addLeafTask(castEvent);
- } else {
- scheduledRequests.addNonLeafTask(castEvent);
- }
- }
- } else if (event.getType() == EventType.T_SCHEDULE_CANCEL) {
- // when a stage is killed, unassigned query unit attmpts are canceled from the scheduler.
- // This event is triggered by TaskAttempt.
- TaskAttemptToSchedulerEvent castedEvent = (TaskAttemptToSchedulerEvent) event;
- scheduledRequests.leafTasks.remove(castedEvent.getTaskAttempt().getId());
- LOG.info(castedEvent.getTaskAttempt().getId() + " is canceled from " + this.getClass().getSimpleName());
- ((TaskAttemptToSchedulerEvent) event).getTaskAttempt().handle(
- new TaskAttemptEvent(castedEvent.getTaskAttempt().getId(), TaskAttemptEventType.TA_SCHEDULE_CANCELED));
- }
- }
-
- @Override
- public void handleTaskRequestEvent(TaskRequestEvent event) {
-
- taskRequests.handle(event);
- int hosts = scheduledRequests.leafTaskHostMapping.size();
-
- // if available cluster resource are large then tasks, the scheduler thread are working immediately.
- if(remainingScheduledObjectNum() > 0 &&
- (remainingScheduledObjectNum() <= hosts || hosts <= taskRequests.size())){
- synchronized (schedulingThread){
- schedulingThread.notifyAll();
- }
- }
- }
-
- @Override
- public int remainingScheduledObjectNum() {
- return scheduledObjectNum;
- }
-
- private class TaskRequests implements EventHandler<TaskRequestEvent> {
- private final LinkedBlockingQueue<TaskRequestEvent> taskRequestQueue =
- new LinkedBlockingQueue<TaskRequestEvent>();
-
- @Override
- public void handle(TaskRequestEvent event) {
- if(LOG.isDebugEnabled()){
- LOG.debug("TaskRequest: " + event.getContainerId() + "," + event.getExecutionBlockId());
- }
-
- if(stopEventHandling.get()) {
- event.getCallback().run(stopTaskRunnerReq);
- return;
- }
- int qSize = taskRequestQueue.size();
- if (qSize != 0 && qSize % 1000 == 0) {
- LOG.info("Size of event-queue in DefaultTaskScheduler is " + qSize);
- }
- int remCapacity = taskRequestQueue.remainingCapacity();
- if (remCapacity < 1000) {
- LOG.warn("Very low remaining capacity in the event-queue "
- + "of DefaultTaskScheduler: " + remCapacity);
- }
-
- taskRequestQueue.add(event);
- }
-
- public void getTaskRequests(final Collection<TaskRequestEvent> taskRequests,
- int num) {
- taskRequestQueue.drainTo(taskRequests, num);
- }
-
- public int size() {
- return taskRequestQueue.size();
- }
- }
-
- /**
- * One worker can have multiple running task runners. <code>HostVolumeMapping</code>
- * describes various information for one worker, including :
- * <ul>
- * <li>host name</li>
- * <li>rack name</li>
- * <li>unassigned tasks for each disk volume</li>
- * <li>last assigned volume id - it can be used for assigning task in a round-robin manner</li>
- * <li>the number of running tasks for each volume</li>
- * </ul>, each task runner and the concurrency number of running tasks for volumes.
- *
- * Here, we identifier a task runner by {@link ContainerId}, and we use volume ids to identify
- * all disks in this node. Actually, each volume is only used to distinguish disks, and we don't
- * know a certain volume id indicates a certain disk. If you want to know volume id, please read the below section.
- *
- * <h3>Volume id</h3>
- * Volume id is an integer. Each volume id identifies each disk volume.
- *
- * This volume id can be obtained from org.apache.hadoop.fs.BlockStorageLocation#getVolumeIds()}. *
- * HDFS cannot give any volume id due to unknown reason and disabled config 'dfs.client.file-block-locations.enabled'.
- * In this case, the volume id will be -1 or other native integer.
- *
- * <h3>See Also</h3>
- * <ul>
- * <li>HDFS-3672 (https://issues.apache.org/jira/browse/HDFS-3672).</li>
- * </ul>
- */
- public class HostVolumeMapping {
- private final String host;
- private final String rack;
- /** A key is disk volume, and a value is a list of tasks to be scheduled. */
- private Map<Integer, LinkedHashSet<TaskAttempt>> unassignedTaskForEachVolume =
- Collections.synchronizedMap(new HashMap<Integer, LinkedHashSet<TaskAttempt>>());
- /** A value is last assigned volume id for each task runner */
- private HashMap<TajoContainerId, Integer> lastAssignedVolumeId = new HashMap<TajoContainerId,
- Integer>();
- /**
- * A key is disk volume id, and a value is the load of this volume.
- * This load is measured by counting how many number of tasks are running.
- *
- * These disk volumes are kept in an order of ascending order of the volume id.
- * In other words, the head volume ids are likely to -1, meaning no given volume id.
- */
- private SortedMap<Integer, Integer> diskVolumeLoads = new TreeMap<Integer, Integer>();
- /** The total number of remain tasks in this host */
- private AtomicInteger remainTasksNum = new AtomicInteger(0);
- public static final int REMOTE = -2;
-
-
- public HostVolumeMapping(String host, String rack){
- this.host = host;
- this.rack = rack;
- }
-
- public synchronized void addTaskAttempt(int volumeId, TaskAttempt attemptId){
- synchronized (unassignedTaskForEachVolume){
- LinkedHashSet<TaskAttempt> list = unassignedTaskForEachVolume.get(volumeId);
- if (list == null) {
- list = new LinkedHashSet<TaskAttempt>();
- unassignedTaskForEachVolume.put(volumeId, list);
- }
- list.add(attemptId);
- }
-
- remainTasksNum.incrementAndGet();
-
- if(!diskVolumeLoads.containsKey(volumeId)) diskVolumeLoads.put(volumeId, 0);
- }
-
- /**
- * Priorities
- * 1. a task list in a volume of host
- * 2. unknown block or Non-splittable task in host
- * 3. remote tasks. unassignedTaskForEachVolume is only contained local task. so it will be null
- */
- public synchronized TaskAttemptId getLocalTask(TajoContainerId containerId) {
- int volumeId;
- TaskAttemptId taskAttemptId = null;
-
- if (!lastAssignedVolumeId.containsKey(containerId)) {
- volumeId = getLowestVolumeId();
- increaseConcurrency(containerId, volumeId);
- } else {
- volumeId = lastAssignedVolumeId.get(containerId);
- }
-
- if (unassignedTaskForEachVolume.size() > 0) {
- int retry = unassignedTaskForEachVolume.size();
- do {
- //clean and get a remaining local task
- taskAttemptId = getAndRemove(volumeId);
- if(!unassignedTaskForEachVolume.containsKey(volumeId)) {
- decreaseConcurrency(containerId);
- if (volumeId > REMOTE) {
- diskVolumeLoads.remove(volumeId);
- }
- }
-
- if (taskAttemptId == null) {
- //reassign next volume
- volumeId = getLowestVolumeId();
- increaseConcurrency(containerId, volumeId);
- retry--;
- } else {
- break;
- }
- } while (retry > 0);
- } else {
- this.remainTasksNum.set(0);
- }
- return taskAttemptId;
- }
-
- public synchronized TaskAttemptId getTaskAttemptIdByRack(String rack) {
- TaskAttemptId taskAttemptId = null;
-
- if (unassignedTaskForEachVolume.size() > 0 && this.rack.equals(rack)) {
- int retry = unassignedTaskForEachVolume.size();
- do {
- //clean and get a remaining task
- int volumeId = getLowestVolumeId();
- taskAttemptId = getAndRemove(volumeId);
- if (taskAttemptId == null) {
- if (volumeId > REMOTE) {
- diskVolumeLoads.remove(volumeId);
- }
- retry--;
- } else {
- break;
- }
- } while (retry > 0);
- }
- return taskAttemptId;
- }
-
- private synchronized TaskAttemptId getAndRemove(int volumeId){
- TaskAttemptId taskAttemptId = null;
- if(!unassignedTaskForEachVolume.containsKey(volumeId)) return taskAttemptId;
-
- LinkedHashSet<TaskAttempt> list = unassignedTaskForEachVolume.get(volumeId);
- if(list != null && list.size() > 0){
- TaskAttempt taskAttempt;
- synchronized (unassignedTaskForEachVolume) {
- Iterator<TaskAttempt> iterator = list.iterator();
- taskAttempt = iterator.next();
- iterator.remove();
- }
-
- this.remainTasksNum.getAndDecrement();
- taskAttemptId = taskAttempt.getId();
- for (DataLocation location : taskAttempt.getTask().getDataLocations()) {
- if (!this.getHost().equals(location.getHost())) {
- HostVolumeMapping volumeMapping = scheduledRequests.leafTaskHostMapping.get(location.getHost());
- if (volumeMapping != null) {
- volumeMapping.removeTaskAttempt(location.getVolumeId(), taskAttempt);
- }
- }
- }
- }
-
- if(list == null || list.isEmpty()) {
- unassignedTaskForEachVolume.remove(volumeId);
- }
- return taskAttemptId;
- }
-
- private synchronized void removeTaskAttempt(int volumeId, TaskAttempt taskAttempt){
- if(!unassignedTaskForEachVolume.containsKey(volumeId)) return;
-
- LinkedHashSet<TaskAttempt> tasks = unassignedTaskForEachVolume.get(volumeId);
-
- if(tasks != null && tasks.size() > 0){
- tasks.remove(taskAttempt);
- remainTasksNum.getAndDecrement();
- } else {
- unassignedTaskForEachVolume.remove(volumeId);
- }
- }
-
- /**
- * Increase the count of running tasks and disk loads for a certain task runner.
- *
- * @param containerId The task runner identifier
- * @param volumeId Volume identifier
- * @return the volume load (i.e., how many running tasks use this volume)
- */
- private synchronized int increaseConcurrency(TajoContainerId containerId, int volumeId) {
-
- int concurrency = 1;
- if (diskVolumeLoads.containsKey(volumeId)) {
- concurrency = diskVolumeLoads.get(volumeId) + 1;
- }
-
- if (volumeId > -1) {
- LOG.info("Assigned host : " + host + ", Volume : " + volumeId + ", Concurrency : " + concurrency);
- } else if (volumeId == -1) {
- // this case is disabled namenode block meta or compressed text file or amazon s3
- LOG.info("Assigned host : " + host + ", Unknown Volume : " + volumeId + ", Concurrency : " + concurrency);
- } else if (volumeId == REMOTE) {
- // this case has processed all block on host and it will be assigned to remote
- LOG.info("Assigned host : " + host + ", Remaining local tasks : " + getRemainingLocalTaskSize()
- + ", Remote Concurrency : " + concurrency);
- }
- diskVolumeLoads.put(volumeId, concurrency);
- lastAssignedVolumeId.put(containerId, volumeId);
- return concurrency;
- }
-
- /**
- * Decrease the count of running tasks of a certain task runner
- */
- private synchronized void decreaseConcurrency(TajoContainerId containerId){
- Integer volumeId = lastAssignedVolumeId.get(containerId);
- if(volumeId != null && diskVolumeLoads.containsKey(volumeId)){
- Integer concurrency = diskVolumeLoads.get(volumeId);
- if(concurrency > 0){
- diskVolumeLoads.put(volumeId, concurrency - 1);
- } else {
- if (volumeId > REMOTE) {
- diskVolumeLoads.remove(volumeId);
- }
- }
- }
- lastAssignedVolumeId.remove(containerId);
- }
-
- /**
- * volume of a host : 0 ~ n
- * compressed task, amazon s3, unKnown volume : -1
- * remote task : -2
- */
- public int getLowestVolumeId(){
- Map.Entry<Integer, Integer> volumeEntry = null;
-
- for (Map.Entry<Integer, Integer> entry : diskVolumeLoads.entrySet()) {
- if(volumeEntry == null) volumeEntry = entry;
-
- if (volumeEntry.getValue() >= entry.getValue()) {
- volumeEntry = entry;
- }
- }
-
- if(volumeEntry != null){
- return volumeEntry.getKey();
- } else {
- return REMOTE;
- }
- }
-
- public boolean isAssigned(TajoContainerId containerId){
- return lastAssignedVolumeId.containsKey(containerId);
- }
-
- public boolean isRemote(TajoContainerId containerId){
- Integer volumeId = lastAssignedVolumeId.get(containerId);
- if(volumeId == null || volumeId > REMOTE){
- return false;
- } else {
- return true;
- }
- }
-
- public int getRemoteConcurrency(){
- return getVolumeConcurrency(REMOTE);
- }
-
- public int getVolumeConcurrency(int volumeId){
- Integer size = diskVolumeLoads.get(volumeId);
- if(size == null) return 0;
- else return size;
- }
-
- public int getRemainingLocalTaskSize(){
- return remainTasksNum.get();
- }
-
- public String getHost() {
-
- return host;
- }
-
- public String getRack() {
- return rack;
- }
- }
-
- private class ScheduledRequests {
- // two list leafTasks and nonLeafTasks keep all tasks to be scheduled. Even though some task is included in
- // leafTaskHostMapping or leafTasksRackMapping, some task T will not be sent to a task runner
- // if the task is not included in leafTasks and nonLeafTasks.
- private final Set<TaskAttemptId> leafTasks = Collections.synchronizedSet(new HashSet<TaskAttemptId>());
- private final Set<TaskAttemptId> nonLeafTasks = Collections.synchronizedSet(new HashSet<TaskAttemptId>());
- private Map<String, HostVolumeMapping> leafTaskHostMapping = Maps.newConcurrentMap();
- private final Map<String, HashSet<TaskAttemptId>> leafTasksRackMapping = Maps.newConcurrentMap();
-
- private synchronized void addLeafTask(TaskAttemptToSchedulerEvent event) {
- TaskAttempt taskAttempt = event.getTaskAttempt();
- List<DataLocation> locations = taskAttempt.getTask().getDataLocations();
-
- for (DataLocation location : locations) {
- String host = location.getHost();
-
- HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host);
- if (hostVolumeMapping == null) {
- String rack = RackResolver.resolve(host).getNetworkLocation();
- hostVolumeMapping = new HostVolumeMapping(host, rack);
- leafTaskHostMapping.put(host, hostVolumeMapping);
- }
- hostVolumeMapping.addTaskAttempt(location.getVolumeId(), taskAttempt);
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Added attempt req to host " + host);
- }
-
- HashSet<TaskAttemptId> list = leafTasksRackMapping.get(hostVolumeMapping.getRack());
- if (list == null) {
- list = new HashSet<TaskAttemptId>();
- leafTasksRackMapping.put(hostVolumeMapping.getRack(), list);
- }
-
- list.add(taskAttempt.getId());
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Added attempt req to rack " + hostVolumeMapping.getRack());
- }
- }
-
- leafTasks.add(taskAttempt.getId());
- }
-
- private void addNonLeafTask(TaskAttemptToSchedulerEvent event) {
- nonLeafTasks.add(event.getTaskAttempt().getId());
- }
-
- public int leafTaskNum() {
- return leafTasks.size();
- }
-
- public int nonLeafTaskNum() {
- return nonLeafTasks.size();
- }
-
- public Set<TaskAttemptId> assignedRequest = new HashSet<TaskAttemptId>();
-
- private TaskAttemptId allocateLocalTask(String host, TajoContainerId containerId){
- HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host);
-
- if (hostVolumeMapping != null) { //tajo host is located in hadoop datanode
- for (int i = 0; i < hostVolumeMapping.getRemainingLocalTaskSize(); i++) {
- TaskAttemptId attemptId = hostVolumeMapping.getLocalTask(containerId);
-
- if(attemptId == null) break;
- //find remaining local task
- if (leafTasks.contains(attemptId)) {
- leafTasks.remove(attemptId);
- //LOG.info(attemptId + " Assigned based on host match " + hostName);
- hostLocalAssigned++;
- totalAssigned++;
- return attemptId;
- }
- }
- }
- return null;
- }
-
- private TaskAttemptId allocateRackTask(String host) {
-
- List<HostVolumeMapping> remainingTasks = Lists.newArrayList(leafTaskHostMapping.values());
- String rack = RackResolver.resolve(host).getNetworkLocation();
- TaskAttemptId attemptId = null;
-
- if (remainingTasks.size() > 0) {
- synchronized (scheduledRequests) {
- //find largest remaining task of other host in rack
- Collections.sort(remainingTasks, new Comparator<HostVolumeMapping>() {
- @Override
- public int compare(HostVolumeMapping v1, HostVolumeMapping v2) {
- // descending remaining tasks
- if (v2.remainTasksNum.get() > v1.remainTasksNum.get()) {
- return 1;
- } else if (v2.remainTasksNum.get() == v1.remainTasksNum.get()) {
- return 0;
- } else {
- return -1;
- }
- }
- });
- }
-
- for (HostVolumeMapping tasks : remainingTasks) {
- for (int i = 0; i < tasks.getRemainingLocalTaskSize(); i++) {
- TaskAttemptId tId = tasks.getTaskAttemptIdByRack(rack);
-
- if (tId == null) break;
-
- if (leafTasks.contains(tId)) {
- leafTasks.remove(tId);
- attemptId = tId;
- break;
- }
- }
- if(attemptId != null) break;
- }
- }
-
- //find task in rack
- if (attemptId == null) {
- HashSet<TaskAttemptId> list = leafTasksRackMapping.get(rack);
- if (list != null) {
- synchronized (list) {
- Iterator<TaskAttemptId> iterator = list.iterator();
- while (iterator.hasNext()) {
- TaskAttemptId tId = iterator.next();
- iterator.remove();
- if (leafTasks.contains(tId)) {
- leafTasks.remove(tId);
- attemptId = tId;
- break;
- }
- }
- }
- }
- }
-
- if (attemptId != null) {
- rackLocalAssigned++;
- totalAssigned++;
-
- LOG.info(String.format("Assigned Local/Rack/Total: (%d/%d/%d), Locality: %.2f%%, Rack host: %s",
- hostLocalAssigned, rackLocalAssigned, totalAssigned,
- ((double) hostLocalAssigned / (double) totalAssigned) * 100, host));
-
- }
- return attemptId;
- }
-
- public void assignToLeafTasks(LinkedList<TaskRequestEvent> taskRequests) {
- Collections.shuffle(taskRequests);
- LinkedList<TaskRequestEvent> remoteTaskRequests = new LinkedList<TaskRequestEvent>();
-
- TaskRequestEvent taskRequest;
- while (leafTasks.size() > 0 && (!taskRequests.isEmpty() || !remoteTaskRequests.isEmpty())) {
- taskRequest = taskRequests.pollFirst();
- if(taskRequest == null) { // if there are only remote task requests
- taskRequest = remoteTaskRequests.pollFirst();
- }
-
- // checking if this container is still alive.
- // If not, ignore the task request and stop the task runner
- ContainerProxy container = context.getMasterContext().getResourceAllocator()
- .getContainer(taskRequest.getContainerId());
- if(container == null) {
- taskRequest.getCallback().run(stopTaskRunnerReq);
- continue;
- }
-
- // getting the hostname of requested node
- WorkerConnectionInfo connectionInfo =
- context.getMasterContext().getResourceAllocator().getWorkerConnectionInfo(taskRequest.getWorkerId());
- String host = connectionInfo.getHost();
-
- // if there are no worker matched to the hostname a task request
- if(!leafTaskHostMapping.containsKey(host)){
- String normalizedHost = NetUtils.normalizeHost(host);
-
- if(!leafTaskHostMapping.containsKey(normalizedHost) && !taskRequests.isEmpty()){
- // this case means one of either cases:
- // * there are no blocks which reside in this node.
- // * all blocks which reside in this node are consumed, and this task runner requests a remote task.
- // In this case, we transfer the task request to the remote task request list, and skip the followings.
- remoteTaskRequests.add(taskRequest);
- continue;
- }
- }
-
- TajoContainerId containerId = taskRequest.getContainerId();
- LOG.debug("assignToLeafTasks: " + taskRequest.getExecutionBlockId() + "," +
- "containerId=" + containerId);
-
- //////////////////////////////////////////////////////////////////////
- // disk or host-local allocation
- //////////////////////////////////////////////////////////////////////
- TaskAttemptId attemptId = allocateLocalTask(host, containerId);
-
- if (attemptId == null) { // if a local task cannot be found
- HostVolumeMapping hostVolumeMapping = leafTaskHostMapping.get(host);
-
- if(hostVolumeMapping != null) {
- if(!hostVolumeMapping.isRemote(containerId)){
- // assign to remote volume
- hostVolumeMapping.decreaseConcurrency(containerId);
- hostVolumeMapping.increaseConcurrency(containerId, HostVolumeMapping.REMOTE);
- }
- // this part is remote concurrency management of a tail tasks
- int tailLimit = Math.max(remainingScheduledObjectNum() / (leafTaskHostMapping.size() * 2), 1);
-
- if(hostVolumeMapping.getRemoteConcurrency() > tailLimit){
- //release container
- hostVolumeMapping.decreaseConcurrency(containerId);
- taskRequest.getCallback().run(stopTaskRunnerReq);
- continue;
- }
- }
-
- //////////////////////////////////////////////////////////////////////
- // rack-local allocation
- //////////////////////////////////////////////////////////////////////
- attemptId = allocateRackTask(host);
-
- //////////////////////////////////////////////////////////////////////
- // random node allocation
- //////////////////////////////////////////////////////////////////////
- if (attemptId == null && leafTaskNum() > 0) {
- synchronized (leafTasks){
- attemptId = leafTasks.iterator().next();
- leafTasks.remove(attemptId);
- rackLocalAssigned++;
- totalAssigned++;
- LOG.info(String.format("Assigned Local/Remote/Total: (%d/%d/%d), Locality: %.2f%%,",
- hostLocalAssigned, rackLocalAssigned, totalAssigned,
- ((double) hostLocalAssigned / (double) totalAssigned) * 100));
- }
- }
- }
-
- if (attemptId != null) {
- Task task = stage.getTask(attemptId.getTaskId());
- TaskRequest taskAssign = new TaskRequestImpl(
- attemptId,
- new ArrayList<FragmentProto>(task.getAllFragments()),
- "",
- false,
- LogicalNodeSerializer.serialize(task.getLogicalPlan()),
- context.getMasterContext().getQueryContext(),
- stage.getDataChannel(), stage.getBlock().getEnforcer());
- if (checkIfInterQuery(stage.getMasterPlan(), stage.getBlock())) {
- taskAssign.setInterQuery();
- }
-
- context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId,
- taskRequest.getContainerId(), connectionInfo));
- assignedRequest.add(attemptId);
-
- scheduledObjectNum--;
- taskRequest.getCallback().run(taskAssign.getProto());
- } else {
- throw new RuntimeException("Illegal State!!!!!!!!!!!!!!!!!!!!!");
- }
- }
- }
-
- private boolean checkIfInterQuery(MasterPlan masterPlan, ExecutionBlock block) {
- if (masterPlan.isRoot(block)) {
- return false;
- }
-
- ExecutionBlock parent = masterPlan.getParent(block);
- if (masterPlan.isRoot(parent) && parent.hasUnion()) {
- return false;
- }
-
- return true;
- }
-
- public void assignToNonLeafTasks(LinkedList<TaskRequestEvent> taskRequests) {
- Collections.shuffle(taskRequests);
-
- TaskRequestEvent taskRequest;
- while (!taskRequests.isEmpty()) {
- taskRequest = taskRequests.pollFirst();
- LOG.debug("assignToNonLeafTasks: " + taskRequest.getExecutionBlockId());
-
- TaskAttemptId attemptId;
- // random allocation
- if (nonLeafTasks.size() > 0) {
- synchronized (nonLeafTasks){
- attemptId = nonLeafTasks.iterator().next();
- nonLeafTasks.remove(attemptId);
- }
- LOG.debug("Assigned based on * match");
-
- Task task;
- task = stage.getTask(attemptId.getTaskId());
- TaskRequest taskAssign = new TaskRequestImpl(
- attemptId,
- Lists.newArrayList(task.getAllFragments()),
- "",
- false,
- LogicalNodeSerializer.serialize(task.getLogicalPlan()),
- context.getMasterContext().getQueryContext(),
- stage.getDataChannel(),
- stage.getBlock().getEnforcer());
- if (checkIfInterQuery(stage.getMasterPlan(), stage.getBlock())) {
- taskAssign.setInterQuery();
- }
- for(Map.Entry<String, Set<FetchImpl>> entry: task.getFetchMap().entrySet()) {
- Collection<FetchImpl> fetches = entry.getValue();
- if (fetches != null) {
- for (FetchImpl fetch : fetches) {
- taskAssign.addFetch(entry.getKey(), fetch);
- }
- }
- }
-
- WorkerConnectionInfo connectionInfo = context.getMasterContext().getResourceAllocator().
- getWorkerConnectionInfo(taskRequest.getWorkerId());
- context.getMasterContext().getEventHandler().handle(new TaskAttemptAssignedEvent(attemptId,
- taskRequest.getContainerId(), connectionInfo));
- taskRequest.getCallback().run(taskAssign.getProto());
- totalAssigned++;
- scheduledObjectNum--;
- }
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/FetchScheduleEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/FetchScheduleEvent.java b/tajo-core/src/main/java/org/apache/tajo/master/FetchScheduleEvent.java
deleted file mode 100644
index 21e376c..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/FetchScheduleEvent.java
+++ /dev/null
@@ -1,40 +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.tajo.master;
-
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.master.event.TaskSchedulerEvent;
-import org.apache.tajo.worker.FetchImpl;
-
-import java.util.List;
-import java.util.Map;
-
-public class FetchScheduleEvent extends TaskSchedulerEvent {
- private final Map<String, List<FetchImpl>> fetches;
-
- public FetchScheduleEvent(final EventType eventType, final ExecutionBlockId blockId,
- final Map<String, List<FetchImpl>> fetches) {
- super(eventType, blockId);
- this.fetches = fetches;
- }
-
- public Map<String, List<FetchImpl>> getFetches() {
- return fetches;
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/FragmentPair.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/FragmentPair.java b/tajo-core/src/main/java/org/apache/tajo/master/FragmentPair.java
deleted file mode 100644
index 827386b..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/FragmentPair.java
+++ /dev/null
@@ -1,73 +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.tajo.master;
-
-import com.google.common.base.Objects;
-import org.apache.tajo.storage.fragment.Fragment;
-
-/**
- * FragmentPair consists of two fragments, a left fragment and a right fragment.
- * According to queries, it can have the different values.
- * For join queries, it is assumed to have both fragments.
- * Also, the left fragment is assumed to be a fragment of the larger table.
- * For other queries, it is assumed to have only a left fragment.
- */
-public class FragmentPair {
- private Fragment leftFragment;
- private Fragment rightFragment;
-
- public FragmentPair(Fragment left) {
- this.leftFragment = left;
- }
-
- public FragmentPair(Fragment left, Fragment right) {
- this.leftFragment = left;
- this.rightFragment = right;
- }
-
- public Fragment getLeftFragment() {
- return leftFragment;
- }
-
- public Fragment getRightFragment() {
- return rightFragment;
- }
-
- @Override
- public boolean equals(Object o) {
- if (o instanceof FragmentPair) {
- FragmentPair other = (FragmentPair) o;
- boolean eq = this.leftFragment.equals(other.leftFragment);
- if (this.rightFragment != null && other.rightFragment != null) {
- eq &= this.rightFragment.equals(other.rightFragment);
- } else if (this.rightFragment == null && other.rightFragment == null) {
- eq &= true;
- } else {
- return false;
- }
- return eq;
- }
- return false;
- }
-
- @Override
- public int hashCode() {
- return Objects.hashCode(leftFragment, rightFragment);
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
index 51964f0..9d853a5 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@ -39,7 +39,7 @@ import org.apache.tajo.master.TajoMaster.MasterContext;
import org.apache.tajo.master.exec.DDLExecutor;
import org.apache.tajo.master.exec.QueryExecutor;
import org.apache.tajo.master.exec.prehook.DistributedQueryHookManager;
-import org.apache.tajo.master.session.Session;
+import org.apache.tajo.session.Session;
import org.apache.tajo.plan.*;
import org.apache.tajo.plan.logical.InsertNode;
import org.apache.tajo.plan.logical.LogicalRootNode;
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultFileScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultFileScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultFileScanner.java
deleted file mode 100644
index d6ea459..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultFileScanner.java
+++ /dev/null
@@ -1,164 +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.tajo.master;
-
-import com.google.protobuf.ByteString;
-
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.TaskAttemptId;
-import org.apache.tajo.TaskId;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableDesc;
-import org.apache.tajo.catalog.proto.CatalogProtos.FragmentProto;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.plan.logical.ScanNode;
-import org.apache.tajo.engine.planner.physical.SeqScanExec;
-import org.apache.tajo.engine.query.QueryContext;
-import org.apache.tajo.storage.RowStoreUtil;
-import org.apache.tajo.storage.RowStoreUtil.RowStoreEncoder;
-import org.apache.tajo.storage.StorageManager;
-import org.apache.tajo.storage.Tuple;
-import org.apache.tajo.storage.fragment.Fragment;
-import org.apache.tajo.storage.fragment.FragmentConvertor;
-import org.apache.tajo.worker.TaskAttemptContext;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-public class NonForwardQueryResultFileScanner implements NonForwardQueryResultScanner {
- private static final int MAX_FRAGMENT_NUM_PER_SCAN = 100;
-
- private QueryId queryId;
- private String sessionId;
- private SeqScanExec scanExec;
- private TableDesc tableDesc;
- private RowStoreEncoder rowEncoder;
- private int maxRow;
- private int currentNumRows;
- private TaskAttemptContext taskContext;
- private TajoConf tajoConf;
- private ScanNode scanNode;
-
- private int currentFragmentIndex = 0;
-
- public NonForwardQueryResultFileScanner(TajoConf tajoConf, String sessionId, QueryId queryId, ScanNode scanNode,
- TableDesc tableDesc, int maxRow) throws IOException {
- this.tajoConf = tajoConf;
- this.sessionId = sessionId;
- this.queryId = queryId;
- this.scanNode = scanNode;
- this.tableDesc = tableDesc;
- this.maxRow = maxRow;
- this.rowEncoder = RowStoreUtil.createEncoder(tableDesc.getLogicalSchema());
- }
-
- public void init() throws IOException {
- initSeqScanExec();
- }
-
- private void initSeqScanExec() throws IOException {
- List<Fragment> fragments = StorageManager.getStorageManager(tajoConf, tableDesc.getMeta().getStoreType())
- .getNonForwardSplit(tableDesc, currentFragmentIndex, MAX_FRAGMENT_NUM_PER_SCAN);
-
- if (fragments != null && !fragments.isEmpty()) {
- FragmentProto[] fragmentProtos = FragmentConvertor.toFragmentProtoArray(fragments.toArray(new Fragment[] {}));
- this.taskContext = new TaskAttemptContext(
- new QueryContext(tajoConf), null,
- new TaskAttemptId(new TaskId(new ExecutionBlockId(queryId, 1), 0), 0),
- fragmentProtos, null);
- try {
- // scanNode must be clone cause SeqScanExec change target in the case of
- // a partitioned table.
- scanExec = new SeqScanExec(taskContext, (ScanNode) scanNode.clone(), fragmentProtos);
- } catch (CloneNotSupportedException e) {
- throw new IOException(e.getMessage(), e);
- }
- scanExec.init();
- currentFragmentIndex += fragments.size();
- }
- }
-
- public QueryId getQueryId() {
- return queryId;
- }
-
- public String getSessionId() {
- return sessionId;
- }
-
- public void setScanExec(SeqScanExec scanExec) {
- this.scanExec = scanExec;
- }
-
- public TableDesc getTableDesc() {
- return tableDesc;
- }
-
- public void close() throws Exception {
- if (scanExec != null) {
- scanExec.close();
- scanExec = null;
- }
- }
-
- public List<ByteString> getNextRows(int fetchRowNum) throws IOException {
- List<ByteString> rows = new ArrayList<ByteString>();
- if (scanExec == null) {
- return rows;
- }
- int rowCount = 0;
- while (true) {
- Tuple tuple = scanExec.next();
- if (tuple == null) {
- scanExec.close();
- scanExec = null;
- initSeqScanExec();
- if (scanExec != null) {
- tuple = scanExec.next();
- }
- if (tuple == null) {
- if (scanExec != null) {
- scanExec.close();
- scanExec = null;
- }
- break;
- }
- }
- rows.add(ByteString.copyFrom((rowEncoder.toBytes(tuple))));
- rowCount++;
- currentNumRows++;
- if (rowCount >= fetchRowNum) {
- break;
- }
- if (currentNumRows >= maxRow) {
- scanExec.close();
- scanExec = null;
- break;
- }
- }
- return rows;
- }
-
- @Override
- public Schema getLogicalSchema() {
- return tableDesc.getLogicalSchema();
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/1c29c1cb/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultScanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultScanner.java b/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultScanner.java
deleted file mode 100644
index 7e7d705..0000000
--- a/tajo-core/src/main/java/org/apache/tajo/master/NonForwardQueryResultScanner.java
+++ /dev/null
@@ -1,46 +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.tajo.master;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.tajo.QueryId;
-import org.apache.tajo.catalog.Schema;
-import org.apache.tajo.catalog.TableDesc;
-
-import com.google.protobuf.ByteString;
-
-public interface NonForwardQueryResultScanner {
-
- public void close() throws Exception;
-
- public Schema getLogicalSchema();
-
- public List<ByteString> getNextRows(int fetchRowNum) throws IOException;
-
- public QueryId getQueryId();
-
- public String getSessionId();
-
- public TableDesc getTableDesc();
-
- public void init() throws IOException;
-
-}