http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/IsNullEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/IsNullEval.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/IsNullEval.java
new file mode 100644
index 0000000..872edf9
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/IsNullEval.java
@@ -0,0 +1,113 @@
+/**
+ * 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.eval;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.datum.BooleanDatum;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.storage.Tuple;
+
+public class IsNullEval extends BinaryEval {
+ private final static ConstEval NULL_EVAL = new ConstEval(DatumFactory.createNullDatum());
+ private static final DataType [] RES_TYPE =
+ CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.BOOLEAN);
+
+ // persistent variables
+ @Expose private boolean isNot;
+ @Expose private Column columnRef;
+ @Expose private Integer fieldId = null;
+
+ public IsNullEval(boolean not, FieldEval field) {
+ super(Type.IS, field, NULL_EVAL);
+ this.isNot = not;
+ this.columnRef = field.getColumnRef();
+ }
+
+ @Override
+ public EvalContext newContext() {
+ return new IsNullEvalCtx();
+ }
+
+ @Override
+ public DataType[] getValueType() {
+ return RES_TYPE;
+ }
+
+ @Override
+ public String getName() {
+ return "?";
+ }
+
+ @Override
+ public void eval(EvalContext ctx, Schema schema, Tuple tuple) {
+ IsNullEvalCtx isNullCtx = (IsNullEvalCtx) ctx;
+ if (fieldId == null) {
+ fieldId = schema.getColumnId(columnRef.getQualifiedName());
+ }
+ if (isNot) {
+ isNullCtx.result.setValue(!(tuple.get(fieldId) instanceof NullDatum));
+ } else {
+ isNullCtx.result.setValue(tuple.get(fieldId) instanceof NullDatum);
+ }
+ }
+
+ @Override
+ public Datum terminate(EvalContext ctx) {
+ return ((IsNullEvalCtx)ctx).result;
+ }
+
+ public boolean isNot() {
+ return isNot;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof IsNullEval) {
+ IsNullEval other = (IsNullEval) obj;
+ return super.equals(other) &&
+ this.columnRef.equals(other.columnRef) &&
+ this.fieldId == other.fieldId;
+ } else {
+ return false;
+ }
+ }
+
+ public Object clone() throws CloneNotSupportedException {
+ IsNullEval isNullEval = (IsNullEval) super.clone();
+ isNullEval.columnRef = (Column) columnRef.clone();
+ isNullEval.fieldId = fieldId;
+
+ return isNullEval;
+ }
+
+ private class IsNullEvalCtx implements EvalContext {
+ BooleanDatum result;
+
+ IsNullEvalCtx() {
+ this.result = DatumFactory.createBool(false);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/LikeEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/LikeEval.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/LikeEval.java
new file mode 100644
index 0000000..4c7cbe8
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/LikeEval.java
@@ -0,0 +1,96 @@
+/**
+ * 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.eval;
+
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.datum.BooleanDatum;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.storage.Tuple;
+
+import java.util.regex.Pattern;
+
+public class LikeEval extends BinaryEval {
+ @Expose private boolean not;
+ @Expose private Column column;
+ @Expose private String pattern;
+ private static final DataType [] RES_TYPE =
+ CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.BOOLEAN);
+
+ // temporal variables
+ private Integer fieldId = null;
+ private Pattern compiled;
+ private BooleanDatum result;
+
+
+ public LikeEval(boolean not, FieldEval field, ConstEval pattern) {
+ super(Type.LIKE, field, pattern);
+ this.not = not;
+ this.column = field.getColumnRef();
+ this.pattern = pattern.getValue().asChars();
+ }
+
+ public void compile(String pattern) {
+ String regex = pattern.replace("?", ".");
+ regex = regex.replace("%", ".*");
+
+ this.compiled = Pattern.compile(regex,
+ Pattern.CASE_INSENSITIVE | Pattern.DOTALL);
+ result = DatumFactory.createBool(false);
+ }
+
+ @Override
+ public DataType [] getValueType() {
+ return RES_TYPE;
+ }
+
+ @Override
+ public String getName() {
+ return "?";
+ }
+
+ @Override
+ public void eval(EvalContext ctx, Schema schema, Tuple tuple) {
+ if (fieldId == null) {
+ fieldId = schema.getColumnId(column.getQualifiedName());
+ compile(this.pattern);
+ }
+ TextDatum str = tuple.getString(fieldId);
+ if (not) {
+ result.setValue(!compiled.matcher(str.asChars()).matches());
+ } else {
+ result.setValue(compiled.matcher(str.asChars()).matches());
+ }
+ }
+
+ public Datum terminate(EvalContext ctx) {
+ return result;
+ }
+
+ @Override
+ public String toString() {
+ return this.column + " like '" + pattern +"'";
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/NotEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/NotEval.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/NotEval.java
new file mode 100644
index 0000000..71fac77
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/NotEval.java
@@ -0,0 +1,117 @@
+/**
+ * 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.eval;
+
+import com.google.common.base.Preconditions;
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.storage.Tuple;
+
+public class NotEval extends EvalNode implements Cloneable {
+ @Expose private EvalNode subExpr;
+ private static final DataType[] RES_TYPE =
+ CatalogUtil.newDataTypesWithoutLen(TajoDataTypes.Type.BOOLEAN);
+
+ public NotEval(EvalNode subExpr) {
+ super(Type.NOT);
+ Preconditions.checkArgument(
+ subExpr instanceof BinaryEval || subExpr instanceof NotEval);
+ this.subExpr = subExpr;
+ }
+
+ @Override
+ public EvalContext newContext() {
+ NotEvalCtx newCtx = new NotEvalCtx();
+ newCtx.subExprCtx = subExpr.newContext();
+ return newCtx;
+ }
+
+ @Override
+ public DataType [] getValueType() {
+ return RES_TYPE;
+ }
+
+ @Override
+ public String getName() {
+ return "?";
+ }
+
+ @Override
+ public void eval(EvalContext ctx, Schema schema, Tuple tuple) {
+ subExpr.eval(((NotEvalCtx)ctx).subExprCtx, schema, tuple);
+ }
+
+ @Override
+ public Datum terminate(EvalContext ctx) {
+ return DatumFactory.createBool(!subExpr.terminate(((NotEvalCtx)ctx).subExprCtx).asBool());
+ }
+
+ @Override
+ public String toString() {
+ return "NOT " + subExpr.toString();
+ }
+
+ @Override
+ public void preOrder(EvalNodeVisitor visitor) {
+ visitor.visit(this);
+ if (subExpr instanceof NotEval) {
+ ((NotEval)subExpr).subExpr.preOrder(visitor);
+ } else {
+ subExpr.leftExpr.preOrder(visitor);
+ subExpr.rightExpr.preOrder(visitor);
+ }
+ }
+
+ @Override
+ public void postOrder(EvalNodeVisitor visitor) {
+ if (subExpr instanceof NotEval) {
+ ((NotEval)subExpr).subExpr.preOrder(visitor);
+ } else {
+ subExpr.leftExpr.preOrder(visitor);
+ subExpr.rightExpr.preOrder(visitor);
+ }
+ visitor.visit(this);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof NotEval) {
+ NotEval other = (NotEval) obj;
+ return this.subExpr.equals(other.subExpr);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public Object clone() throws CloneNotSupportedException {
+ NotEval eval = (NotEval) super.clone();
+ eval.subExpr = (EvalNode) this.subExpr.clone();
+ return eval;
+ }
+
+ private class NotEvalCtx implements EvalContext {
+ EvalContext subExprCtx;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/PartialBinaryExpr.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/PartialBinaryExpr.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/PartialBinaryExpr.java
new file mode 100644
index 0000000..d43c629
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/PartialBinaryExpr.java
@@ -0,0 +1,82 @@
+/**
+ * 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.eval;
+
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.storage.Tuple;
+
+public class PartialBinaryExpr extends EvalNode {
+
+ public PartialBinaryExpr(Type type) {
+ super(type);
+ }
+
+ public PartialBinaryExpr(Type type, EvalNode left, EvalNode right) {
+ super(type, left, right);
+ }
+
+ @Override
+ public EvalContext newContext() {
+ return null;
+ }
+
+ @Override
+ public DataType[] getValueType() {
+ return null;
+ }
+
+ @Override
+ public String getName() {
+ return "nonamed";
+ }
+
+ @Override
+ public void eval(EvalContext ctx, Schema schema, Tuple tuple) {
+ throw new InvalidEvalException(
+ "ERROR: the partial binary expression cannot be evluated: "
+ + this.toString() );
+ }
+
+ @Override
+ public Datum terminate(EvalContext ctx) {
+ throw new InvalidEvalException(
+ "ERROR: the partial binary expression cannot be terminated: "
+ + this.toString() );
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof PartialBinaryExpr) {
+ PartialBinaryExpr other = (PartialBinaryExpr) obj;
+ return type.equals(other.type) &&
+ leftExpr.equals(other.leftExpr) &&
+ rightExpr.equals(other.rightExpr);
+ }
+ return false;
+ }
+
+ public String toString() {
+ return
+ (leftExpr != null ? leftExpr.toString() : "[EMPTY]")
+ + " " + type + " "
+ + (rightExpr != null ? rightExpr.toString() : "[EMPTY]");
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/EmptyClusterException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/EmptyClusterException.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/EmptyClusterException.java
new file mode 100644
index 0000000..0c7795c
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/EmptyClusterException.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.engine.exception;
+
+public class EmptyClusterException extends Exception {
+
+ public EmptyClusterException() {
+
+ }
+
+ public EmptyClusterException(String msg) {
+ super(msg);
+ }
+
+ public EmptyClusterException(Exception e) {
+ super(e);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/IllegalQueryStatusException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/IllegalQueryStatusException.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/IllegalQueryStatusException.java
new file mode 100644
index 0000000..66e9ee7
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/IllegalQueryStatusException.java
@@ -0,0 +1,38 @@
+/**
+ * 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.exception;
+
+public class IllegalQueryStatusException extends Exception {
+
+ public IllegalQueryStatusException() {
+
+ }
+
+ public IllegalQueryStatusException(String msg) {
+ super(msg);
+ }
+
+ public IllegalQueryStatusException(Exception e) {
+ super(e);
+ }
+
+ public IllegalQueryStatusException(String msg, Exception e) {
+ super(msg, e);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/NTAQueryException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/NTAQueryException.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/NTAQueryException.java
new file mode 100644
index 0000000..f280e50
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/NTAQueryException.java
@@ -0,0 +1,42 @@
+/**
+ * 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.exception;
+
+import java.io.IOException;
+
+public class NTAQueryException extends IOException {
+ private static final long serialVersionUID = -5012296598261064705L;
+
+ public NTAQueryException() {
+ }
+
+ public NTAQueryException(Exception e) {
+ super(e);
+ }
+
+ /**
+ * @param query
+ */
+ public NTAQueryException(String query) {
+ super(query);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/NoSuchQueryIdException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/NoSuchQueryIdException.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/NoSuchQueryIdException.java
new file mode 100644
index 0000000..153164b
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/NoSuchQueryIdException.java
@@ -0,0 +1,43 @@
+/**
+ * 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.exception;
+
+public class NoSuchQueryIdException extends Exception {
+
+ private static final long serialVersionUID = -4425982532461186746L;
+
+ public NoSuchQueryIdException() {
+
+ }
+
+ public NoSuchQueryIdException(String message) {
+ super(message);
+ }
+
+ public NoSuchQueryIdException(Exception e) {
+ super(e);
+ }
+
+ public NoSuchQueryIdException(String message, Exception e) {
+ super(message, e);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/UnfinishedTaskException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/UnfinishedTaskException.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/UnfinishedTaskException.java
new file mode 100644
index 0000000..0f9c47f
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/UnfinishedTaskException.java
@@ -0,0 +1,35 @@
+/**
+ * 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.exception;
+
+public class UnfinishedTaskException extends Exception {
+
+ private static final long serialVersionUID = -3229141373378209229L;
+
+ public UnfinishedTaskException() {
+
+ }
+
+ public UnfinishedTaskException(String message) {
+ super(message);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/UnknownWorkerException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/UnknownWorkerException.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/UnknownWorkerException.java
new file mode 100644
index 0000000..c524b1d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/exception/UnknownWorkerException.java
@@ -0,0 +1,44 @@
+/**
+ * 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.exception;
+
+public class UnknownWorkerException extends Exception {
+
+ /**
+ *
+ */
+ private static final long serialVersionUID = -3677733092100608744L;
+ private String unknownName;
+
+ public UnknownWorkerException(String unknownName) {
+ this.unknownName = unknownName;
+ }
+
+ public UnknownWorkerException(String unknownName, Exception e) {
+ super(e);
+ this.unknownName = unknownName;
+ }
+
+ public String getUnknownName() {
+ return this.unknownName;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/Country.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/Country.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/Country.java
new file mode 100644
index 0000000..fe0c6c1
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/Country.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.engine.function;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.GeneralFunction;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.TextDatum;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.GeoUtil;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
+
+public class Country extends GeneralFunction<TextDatum> {
+
+ public Country() {
+ super(new Column[] {new Column("addr", TEXT)});
+ }
+
+ @Override
+ public Datum eval(Tuple params) {
+ return new TextDatum(GeoUtil.getCountryCode(params.get(0).asChars()));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/InCountry.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/InCountry.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/InCountry.java
new file mode 100644
index 0000000..75c52f1
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/InCountry.java
@@ -0,0 +1,44 @@
+/**
+ * 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;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.GeneralFunction;
+import org.apache.tajo.common.TajoDataTypes;
+import org.apache.tajo.datum.BooleanDatum;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.storage.Tuple;
+import org.apache.tajo.util.GeoUtil;
+
+public class InCountry extends GeneralFunction<BooleanDatum> {
+
+ public InCountry() {
+ super(new Column[] {new Column("addr", TajoDataTypes.Type.TEXT),
+ new Column("code", TajoDataTypes.Type.TEXT)});
+ }
+
+ @Override
+ public Datum eval(Tuple params) {
+ String addr = params.get(0).asChars();
+ String otherCode = params.get(1).asChars();
+ String thisCode = GeoUtil.getCountryCode(addr);
+
+ return new BooleanDatum(thisCode.equals(otherCode));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
new file mode 100644
index 0000000..44ad3b1
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgDouble.java
@@ -0,0 +1,86 @@
+/**
+ * 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.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.ArrayDatum;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.storage.Tuple;
+
+public class AvgDouble extends AggFunction {
+ public AvgDouble() {
+ super(new Column[] {
+ new Column("val", Type.FLOAT8)
+ });
+ }
+
+ public AvgContext newContext() {
+ return new AvgContext();
+ }
+
+ public void init() {
+ }
+
+ @Override
+ public void eval(FunctionContext ctx, Tuple params) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ avgCtx.sum += params.get(0).asFloat8();
+ avgCtx.count++;
+ }
+
+ @Override
+ public void merge(FunctionContext ctx, Tuple part) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ ArrayDatum array = (ArrayDatum) part.get(0);
+ avgCtx.sum += array.get(0).asFloat8();
+ avgCtx.count += array.get(1).asInt8();
+ }
+
+ @Override
+ public Datum getPartialResult(FunctionContext ctx) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ ArrayDatum part = new ArrayDatum(2);
+ part.put(0, DatumFactory.createFloat8(avgCtx.sum));
+ part.put(1, DatumFactory.createInt8(avgCtx.count));
+
+ return part;
+ }
+
+ @Override
+ public DataType[] getPartialResultType() {
+ return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8, Type.INT8);
+ }
+
+ @Override
+ public Datum terminate(FunctionContext ctx) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ return DatumFactory.createFloat8(avgCtx.sum / avgCtx.count);
+ }
+
+ private class AvgContext implements FunctionContext {
+ double sum;
+ long count;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgFloat.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgFloat.java
new file mode 100644
index 0000000..c1ac80c
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgFloat.java
@@ -0,0 +1,85 @@
+/**
+ * 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.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.ArrayDatum;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Float4Datum;
+import org.apache.tajo.storage.Tuple;
+
+public class AvgFloat extends AggFunction<Float4Datum> {
+
+ public AvgFloat() {
+ super(new Column[] {
+ new Column("val", Type.FLOAT8)
+ });
+ }
+
+ public AvgContext newContext() {
+ return new AvgContext();
+ }
+
+ @Override
+ public void eval(FunctionContext ctx, Tuple params) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ avgCtx.sum += params.get(0).asFloat4();
+ avgCtx.count++;
+ }
+
+ @Override
+ public void merge(FunctionContext ctx, Tuple part) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ ArrayDatum array = (ArrayDatum) part.get(0);
+ avgCtx.sum += array.get(0).asFloat8();
+ avgCtx.count += array.get(1).asInt8();
+ }
+
+ @Override
+ public Datum getPartialResult(FunctionContext ctx) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ ArrayDatum part = new ArrayDatum(2);
+ part.put(0, DatumFactory.createFloat8(avgCtx.sum));
+ part.put(1, DatumFactory.createInt8(avgCtx.count));
+
+ return part;
+ }
+
+ @Override
+ public DataType[] getPartialResultType() {
+ return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8, Type.INT8);
+ }
+
+ @Override
+ public Float4Datum terminate(FunctionContext ctx) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ return DatumFactory.createFloat4((float) (avgCtx.sum / avgCtx.count));
+ }
+
+ private class AvgContext implements FunctionContext {
+ double sum;
+ long count;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgInt.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgInt.java
new file mode 100644
index 0000000..f7f528e
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgInt.java
@@ -0,0 +1,85 @@
+/**
+ * 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.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.ArrayDatum;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Float4Datum;
+import org.apache.tajo.storage.Tuple;
+
+public class AvgInt extends AggFunction<Float4Datum> {
+
+ public AvgInt() {
+ super(new Column[] {
+ new Column("val", Type.FLOAT8)
+ });
+ }
+
+ public AvgContext newContext() {
+ return new AvgContext();
+ }
+
+ @Override
+ public void eval(FunctionContext ctx, Tuple params) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ avgCtx.sum += params.get(0).asInt4();
+ avgCtx.count++;
+ }
+
+ @Override
+ public void merge(FunctionContext ctx, Tuple part) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ ArrayDatum array = (ArrayDatum) part.get(0);
+ avgCtx.sum += array.get(0).asInt8();
+ avgCtx.count += array.get(1).asInt8();
+ }
+
+ @Override
+ public Datum getPartialResult(FunctionContext ctx) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ ArrayDatum part = new ArrayDatum(2);
+ part.put(0, DatumFactory.createInt8(avgCtx.sum));
+ part.put(1, DatumFactory.createInt8(avgCtx.count));
+
+ return part;
+ }
+
+ @Override
+ public DataType [] getPartialResultType() {
+ return CatalogUtil.newDataTypesWithoutLen(Type.INT8, Type.INT8);
+ }
+
+ @Override
+ public Float4Datum terminate(FunctionContext ctx) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ return DatumFactory.createFloat4((float) avgCtx.sum / avgCtx.count);
+ }
+
+ private class AvgContext implements FunctionContext {
+ long sum;
+ long count;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgLong.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgLong.java
new file mode 100644
index 0000000..bc7e984
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/AvgLong.java
@@ -0,0 +1,85 @@
+/**
+ * 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.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.ArrayDatum;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Float8Datum;
+import org.apache.tajo.storage.Tuple;
+
+public class AvgLong extends AggFunction<Float8Datum> {
+
+ public AvgLong() {
+ super(new Column[] {
+ new Column("val", Type.FLOAT8)
+ });
+ }
+
+ public AvgContext newContext() {
+ return new AvgContext();
+ }
+
+ @Override
+ public void eval(FunctionContext ctx, Tuple params) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ avgCtx.sum += params.get(0).asInt8();
+ avgCtx.count++;
+ }
+
+ @Override
+ public void merge(FunctionContext ctx, Tuple part) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ ArrayDatum array = (ArrayDatum) part.get(0);
+ avgCtx.sum += array.get(0).asInt8();
+ avgCtx.count += array.get(1).asInt8();
+ }
+
+ @Override
+ public Datum getPartialResult(FunctionContext ctx) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ ArrayDatum part = new ArrayDatum(2);
+ part.put(0, DatumFactory.createInt8(avgCtx.sum));
+ part.put(1, DatumFactory.createInt8(avgCtx.count));
+
+ return part;
+ }
+
+ @Override
+ public DataType[] getPartialResultType() {
+ return CatalogUtil.newDataTypesWithoutLen(Type.INT8, Type.INT8);
+ }
+
+ @Override
+ public Float8Datum terminate(FunctionContext ctx) {
+ AvgContext avgCtx = (AvgContext) ctx;
+ return DatumFactory.createFloat8((double) avgCtx.sum / avgCtx.count);
+ }
+
+ private class AvgContext implements FunctionContext {
+ long sum;
+ long count;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/CountRows.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/CountRows.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/CountRows.java
new file mode 100644
index 0000000..41d27e0
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/CountRows.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.engine.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+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.storage.Tuple;
+
+public class CountRows extends AggFunction<Datum> {
+
+ public CountRows() {
+ super(NoArgs);
+ }
+
+ protected CountRows(Column[] columns) {
+ super(columns);
+ }
+
+ @Override
+ public FunctionContext newContext() {
+ return new CountRowContext();
+ }
+
+ @Override
+ public void eval(FunctionContext ctx, Tuple params) {
+ ((CountRowContext) ctx).count++;
+ }
+
+ @Override
+ public void merge(FunctionContext ctx, Tuple part) {
+ ((CountRowContext) ctx).count += part.get(0).asInt8();
+ }
+
+ @Override
+ public Datum getPartialResult(FunctionContext ctx) {
+ return DatumFactory.createInt8(((CountRowContext) ctx).count);
+ }
+
+ @Override
+ public DataType[] getPartialResultType() {
+ return CatalogUtil.newDataTypesWithoutLen(Type.INT8);
+ }
+
+ @Override
+ public Int8Datum terminate(FunctionContext ctx) {
+ return DatumFactory.createInt8(((CountRowContext) ctx).count);
+ }
+
+ protected class CountRowContext implements FunctionContext {
+ long count;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/CountValue.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/CountValue.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/CountValue.java
new file mode 100644
index 0000000..46ccc3b
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/CountValue.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.engine.function.builtin;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.FunctionContext;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.storage.Tuple;
+
+public final class CountValue extends CountRows {
+
+ public CountValue() {
+ super(new Column[] {
+ new Column("col", Type.ANY)
+ });
+ }
+ @Override
+ public void eval(FunctionContext ctx, Tuple params) {
+ if (params.get(0) instanceof NullDatum) {
+ ((CountRowContext) ctx).count++;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/Date.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/Date.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/Date.java
new file mode 100644
index 0000000..e3f4af6
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/Date.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.engine.function.builtin;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.GeneralFunction;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.Int8Datum;
+import org.apache.tajo.storage.Tuple;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.TEXT;
+
+public class Date extends GeneralFunction<Int8Datum> {
+ private final Log LOG = LogFactory.getLog(Date.class);
+ private final static String dateFormat = "dd/MM/yyyy HH:mm:ss";
+
+ public Date() {
+ super(new Column[] {new Column("val", TEXT)});
+ }
+
+ @Override
+ public Datum eval(Tuple params) {
+ try {
+ return DatumFactory.createInt8(new SimpleDateFormat(dateFormat)
+ .parse(params.get(0).asChars()).getTime());
+ } catch (ParseException e) {
+ LOG.error(e);
+ return null;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxDouble.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxDouble.java
new file mode 100644
index 0000000..419362e
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxDouble.java
@@ -0,0 +1,69 @@
+/**
+ * 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.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+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.storage.Tuple;
+
+public class MaxDouble extends AggFunction<Float8Datum> {
+
+ public MaxDouble() {
+ super(new Column[] {
+ new Column("val", Type.FLOAT8)
+ });
+ }
+
+ @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.newDataTypesWithoutLen(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/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxFloat.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxFloat.java
new file mode 100644
index 0000000..9a85faf
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxFloat.java
@@ -0,0 +1,67 @@
+/**
+ * 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.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+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.storage.Tuple;
+
+public class MaxFloat extends AggFunction<Datum> {
+ public MaxFloat() {
+ super(new Column[] {
+ new Column("val", Type.FLOAT8)
+ });
+ }
+
+ @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.newDataTypesWithoutLen(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/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxInt.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxInt.java
new file mode 100644
index 0000000..34ef0f5
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxInt.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.engine.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+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.storage.Tuple;
+
+public class MaxInt extends AggFunction<Datum> {
+
+ public MaxInt() {
+ super(new Column[] {
+ new Column("val", Type.INT8)
+ });
+ }
+
+ @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.newDataTypesWithoutLen(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/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxLong.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxLong.java
new file mode 100644
index 0000000..1364cf1
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MaxLong.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.engine.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+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.storage.Tuple;
+
+public class MaxLong extends AggFunction<Int8Datum> {
+ public MaxLong() {
+ super(new Column[] {
+ new Column("val", Type.INT8)
+ });
+ }
+
+ @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.newDataTypesWithoutLen(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/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinDouble.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinDouble.java
new file mode 100644
index 0000000..5a7a604
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinDouble.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.engine.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+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.storage.Tuple;
+
+public class MinDouble extends AggFunction<Datum> {
+
+ public MinDouble() {
+ super(new Column[] {
+ new Column("val", Type.FLOAT8)
+ });
+ }
+
+ @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.newDataTypesWithoutLen(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/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinFloat.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinFloat.java
new file mode 100644
index 0000000..302a586
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinFloat.java
@@ -0,0 +1,69 @@
+/**
+ * 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.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+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.storage.Tuple;
+
+public class MinFloat extends AggFunction<Float4Datum> {
+
+ public MinFloat() {
+ super(new Column[] {
+ new Column("val", Type.FLOAT4)
+ });
+ }
+
+ @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.newDataTypesWithoutLen(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/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinInt.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinInt.java
new file mode 100644
index 0000000..4aff484
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinInt.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.engine.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+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.storage.Tuple;
+
+public class MinInt extends AggFunction<Datum> {
+
+ public MinInt() {
+ super(new Column[] {
+ new Column("val", Type.INT8)
+ });
+ }
+
+ @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.newDataTypesWithoutLen(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/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinLong.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinLong.java
new file mode 100644
index 0000000..932b88e
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinLong.java
@@ -0,0 +1,69 @@
+/**
+ * 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.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+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.storage.Tuple;
+
+public class MinLong extends AggFunction<Datum> {
+
+ public MinLong() {
+ super(new Column[] {
+ new Column("val", Type.INT8)
+ });
+ }
+
+ @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.newDataTypesWithoutLen(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/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinString.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinString.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinString.java
new file mode 100644
index 0000000..4a5f258
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/MinString.java
@@ -0,0 +1,73 @@
+/**
+ * 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.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+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.storage.Tuple;
+
+public class MinString extends AggFunction<Datum> {
+
+ public MinString() {
+ super(new Column[] {
+ new Column("val", Type.TEXT)
+ });
+ }
+
+ @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.newDataTypesWithoutLen(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/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/RandomInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/RandomInt.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/RandomInt.java
new file mode 100644
index 0000000..3ffddfe
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/RandomInt.java
@@ -0,0 +1,45 @@
+/**
+ * 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.catalog.function.GeneralFunction;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.storage.Tuple;
+
+import java.util.Random;
+
+import static org.apache.tajo.common.TajoDataTypes.Type.INT4;
+
+public class RandomInt extends GeneralFunction<Datum> {
+ private Random random;
+
+ public RandomInt() {
+ super(new Column[] {
+ new Column("val", INT4)
+ });
+ random = new Random(System.nanoTime());
+ }
+
+ @Override
+ public Datum eval(Tuple params) {
+ return DatumFactory.createInt4(random.nextInt(params.get(0).asInt4()));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/SumDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/SumDouble.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/SumDouble.java
new file mode 100644
index 0000000..277a3bf
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/SumDouble.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.engine.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+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.storage.Tuple;
+
+public class SumDouble extends AggFunction<Datum> {
+
+ public SumDouble() {
+ super(new Column[] {
+ new Column("val", Type.FLOAT8)
+ });
+ }
+
+ @Override
+ public FunctionContext newContext() {
+ return new SumContext();
+ }
+
+ @Override
+ public void eval(FunctionContext ctx, Tuple params) {
+ ((SumContext)ctx).sum += params.get(0).asFloat8();
+ }
+
+ @Override
+ public Datum getPartialResult(FunctionContext ctx) {
+ return DatumFactory.createFloat8(((SumContext) ctx).sum);
+ }
+
+ @Override
+ public DataType [] getPartialResultType() {
+ return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT8);
+ }
+
+ @Override
+ public Float8Datum terminate(FunctionContext ctx) {
+ return DatumFactory.createFloat8(((SumContext) ctx).sum);
+ }
+
+ private class SumContext implements FunctionContext {
+ double sum;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/SumFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/SumFloat.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/SumFloat.java
new file mode 100644
index 0000000..251c98d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/SumFloat.java
@@ -0,0 +1,66 @@
+/**
+ * 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.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+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.storage.Tuple;
+
+public class SumFloat extends AggFunction<Datum> {
+ public SumFloat() {
+ super(new Column[] {
+ new Column("val", Type.FLOAT4)
+ });
+ }
+
+ @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.createFloat4(((SumContext) ctx).sum);
+ }
+
+ @Override
+ public DataType [] getPartialResultType() {
+ return CatalogUtil.newDataTypesWithoutLen(Type.FLOAT4);
+ }
+
+ @Override
+ public Datum terminate(FunctionContext ctx) {
+ return DatumFactory.createFloat4(((SumContext) ctx).sum);
+ }
+
+ private class SumContext implements FunctionContext {
+ private float sum;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/SumInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/SumInt.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/SumInt.java
new file mode 100644
index 0000000..c1e607b
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/function/builtin/SumInt.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.engine.function.builtin;
+
+import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.function.AggFunction;
+import org.apache.tajo.catalog.function.FunctionContext;
+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.storage.Tuple;
+
+public class SumInt extends AggFunction<Datum> {
+
+ public SumInt() {
+ super(new Column[] {
+ new Column("val", 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.createInt4(((SumIntContext) ctx).sum);
+ }
+
+ @Override
+ public DataType [] getPartialResultType() {
+ return CatalogUtil.newDataTypesWithoutLen(Type.INT4);
+ }
+
+ @Override
+ public Datum terminate(FunctionContext ctx) {
+ return DatumFactory.createInt4(((SumIntContext) ctx).sum);
+ }
+
+ private class SumIntContext implements FunctionContext {
+ int sum;
+ }
+}
|