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/03/30 11:57:38 UTC

[4/5] tajo git commit: TAJO-921: Add STDDEV_SAMP and STDDEV_POP window functions. (Keuntae Park)

TAJO-921: Add STDDEV_SAMP and STDDEV_POP window functions. (Keuntae Park)

Closes #427


Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/652e4db7
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/652e4db7
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/652e4db7

Branch: refs/heads/index_support
Commit: 652e4db7965e6452ad2a9778f8383141bb63e22f
Parents: 7f77593
Author: Keuntae Park <si...@apache.org>
Authored: Mon Mar 30 15:18:35 2015 +0900
Committer: Keuntae Park <si...@apache.org>
Committed: Mon Mar 30 15:18:35 2015 +0900

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 .../tajo/engine/function/builtin/StdDev.java    |  94 ++++++++++++++
 .../tajo/engine/function/builtin/StdDevPop.java |  42 ++++++
 .../function/builtin/StdDevPopDouble.java       |  39 ++++++
 .../engine/function/builtin/StdDevPopFloat.java |  39 ++++++
 .../engine/function/builtin/StdDevPopInt.java   |  39 ++++++
 .../engine/function/builtin/StdDevPopLong.java  |  39 ++++++
 .../engine/function/builtin/StdDevSamp.java     |  41 ++++++
 .../function/builtin/StdDevSampDouble.java      |  39 ++++++
 .../function/builtin/StdDevSampFloat.java       |  39 ++++++
 .../engine/function/builtin/StdDevSampInt.java  |  39 ++++++
 .../engine/function/builtin/StdDevSampLong.java |  39 ++++++
 tajo-core/src/main/proto/InternalTypes.proto    |   6 +
 .../engine/function/TestBuiltinFunctions.java   | 129 +++++++++++++++++++
 .../tajo/engine/query/TestWindowQuery.java      |  28 +++-
 .../queries/TestWindowQuery/firstValue1.sql     |  18 ---
 .../queries/TestWindowQuery/lastValue1.sql      |  18 ---
 .../queries/TestWindowQuery/rowNumber1.sql      |   5 -
 .../queries/TestWindowQuery/rowNumber2.sql      |   5 -
 .../queries/TestWindowQuery/rowNumber3.sql      |   7 -
 .../queries/TestWindowQuery/testFirstValue1.sql |  18 +++
 .../queries/TestWindowQuery/testLastValue1.sql  |  18 +++
 .../queries/TestWindowQuery/testRowNumber1.sql  |   5 +
 .../queries/TestWindowQuery/testRowNumber2.sql  |   5 +
 .../queries/TestWindowQuery/testRowNumber3.sql  |   7 +
 .../queries/TestWindowQuery/testStdDevPop1.sql  |  14 ++
 .../queries/TestWindowQuery/testStdDevSamp1.sql |  14 ++
 .../results/TestWindowQuery/firstValue1.result  |   7 -
 .../results/TestWindowQuery/lastValue1.result   |   7 -
 .../results/TestWindowQuery/rowNumber1.result   |   7 -
 .../results/TestWindowQuery/rowNumber2.result   |   7 -
 .../results/TestWindowQuery/rowNumber3.result   |   7 -
 .../TestWindowQuery/testFirstValue1.result      |   7 +
 .../TestWindowQuery/testLastValue1.result       |   7 +
 .../TestWindowQuery/testRowNumber1.result       |   7 +
 .../TestWindowQuery/testRowNumber2.result       |   7 +
 .../TestWindowQuery/testRowNumber3.result       |   7 +
 .../TestWindowQuery/testStdDevPop1.result       |   7 +
 .../TestWindowQuery/testStdDevSamp1.result      |   7 +
 39 files changed, 777 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 3e849b4..bf6cbcd 100644
--- a/CHANGES
+++ b/CHANGES
@@ -4,6 +4,8 @@ Release 0.11.0 - unreleased
 
   NEW FEATURES
 
+    TAJO-921: Add STDDEV_SAMP and STDDEV_POP window functions. (Keuntae Park)
+
     TAJO-1135: Implement queryable virtual table for cluster information.
     (jihun)
 

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDev.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDev.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDev.java
new file mode 100644
index 0000000..e736167
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDev.java
@@ -0,0 +1,94 @@
+/**
+ * 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.common.TajoDataTypes.DataType;
+import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.datum.ProtobufDatum;
+import org.apache.tajo.plan.function.AggFunction;
+import org.apache.tajo.plan.function.FunctionContext;
+import org.apache.tajo.storage.Tuple;
+
+import static org.apache.tajo.InternalTypes.StdDevProto;
+
+public abstract class StdDev extends AggFunction<Datum> {
+
+  public StdDev(Column[] definedArgs) {
+    super(definedArgs);
+  }
+
+  public StdDevContext newContext() {
+    return new StdDevContext();
+  }
+
+  @Override
+  public void eval(FunctionContext ctx, Tuple params) {
+    StdDevContext StdDevCtx = (StdDevContext) ctx;
+    Datum datum = params.get(0);
+    if (datum.isNotNull()) {
+      double delta = datum.asFloat8() - StdDevCtx.avg;
+      StdDevCtx.count++;
+      StdDevCtx.avg += delta/StdDevCtx.count;
+      StdDevCtx.squareSumOfDiff += delta * (datum.asFloat8() - StdDevCtx.avg);
+    }
+  }
+
+  @Override
+  public void merge(FunctionContext ctx, Tuple part) {
+    StdDevContext StdDevCtx = (StdDevContext) ctx;
+    Datum d = part.get(0);
+    if (d instanceof NullDatum) {
+      return;
+    }
+    ProtobufDatum datum = (ProtobufDatum) d;
+    StdDevProto proto = (StdDevProto) datum.get();
+    double delta = proto.getAvg() - StdDevCtx.avg;
+    StdDevCtx.avg += delta * proto.getCount() / (StdDevCtx.count + proto.getCount());
+    StdDevCtx.squareSumOfDiff += proto.getSquareSumOfDiff() + delta * delta * StdDevCtx.count * proto.getCount() / (StdDevCtx.count + proto.getCount());
+    StdDevCtx.count += proto.getCount();
+  }
+
+  @Override
+  public Datum getPartialResult(FunctionContext ctx) {
+    StdDevContext StdDevCtx = (StdDevContext) ctx;
+    if (StdDevCtx.count == 0) {
+      return NullDatum.get();
+    }
+    StdDevProto.Builder builder = StdDevProto.newBuilder();
+    builder.setSquareSumOfDiff(StdDevCtx.squareSumOfDiff);
+    builder.setAvg(StdDevCtx.avg);
+    builder.setCount(StdDevCtx.count);
+    return new ProtobufDatum(builder.build());
+  }
+
+  @Override
+  public DataType getPartialResultType() {
+    return CatalogUtil.newDataType(Type.PROTOBUF, StdDevProto.class.getName());
+  }
+
+  protected static class StdDevContext implements FunctionContext {
+    double squareSumOfDiff = 0.0;
+    double avg = 0.0;
+    long count = 0;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPop.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPop.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPop.java
new file mode 100644
index 0000000..3403df1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPop.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.function.builtin;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.plan.function.FunctionContext;
+
+public abstract class StdDevPop extends StdDev {
+  public StdDevPop(Column[] definedArgs) {
+    super(definedArgs);
+  }
+
+  @Override
+  public Datum terminate(FunctionContext ctx) {
+    StdDevContext StdDevCtx = (StdDevContext) ctx;
+    if (StdDevCtx.count == 0) {
+      return NullDatum.get();
+    } else if (StdDevCtx.count == 1) {
+      return DatumFactory.createFloat8(0);
+    }
+    return DatumFactory.createFloat8(Math.sqrt(StdDevCtx.squareSumOfDiff / StdDevCtx.count));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopDouble.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopDouble.java
new file mode 100644
index 0000000..ea1e10b
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopDouble.java
@@ -0,0 +1,39 @@
+/**
+ * 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.common.TajoDataTypes.Type;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+
+@Description(
+    functionName = "STDDEV_POP",
+    description = "The population standard deviation of a set of numbers.",
+    example = "> SELECT STDDEV_POP(expr);",
+    returnType = Type.FLOAT8,
+    paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT8})}
+)
+public class StdDevPopDouble extends StdDevPop {
+  public StdDevPopDouble() {
+    super(new Column[] {
+        new Column("expr", Type.FLOAT8)
+    });
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopFloat.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopFloat.java
new file mode 100644
index 0000000..ea620f6
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopFloat.java
@@ -0,0 +1,39 @@
+/**
+ * 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.common.TajoDataTypes.Type;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+
+@Description(
+    functionName = "STDDEV_POP",
+    description = "The population standard deviation of a set of numbers.",
+    example = "> SELECT STDDEV_POP(expr);",
+    returnType = Type.FLOAT8,
+    paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT4})}
+)
+public class StdDevPopFloat extends StdDevPop {
+  public StdDevPopFloat() {
+    super(new Column[] {
+        new Column("expr", Type.FLOAT4)
+    });
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopInt.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopInt.java
new file mode 100644
index 0000000..d4de6d1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopInt.java
@@ -0,0 +1,39 @@
+/**
+ * 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.common.TajoDataTypes.Type;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+
+@Description(
+    functionName = "STDDEV_POP",
+    description = "The population standard deviation of a set of numbers.",
+    example = "> SELECT STDDEV_POP(expr);",
+    returnType = Type.FLOAT8,
+    paramTypes = {@ParamTypes(paramTypes = {Type.INT4})}
+)
+public class StdDevPopInt extends StdDevPop {
+  public StdDevPopInt() {
+    super(new Column[] {
+        new Column("expr", Type.INT4)
+    });
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopLong.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopLong.java
new file mode 100644
index 0000000..a20f540
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevPopLong.java
@@ -0,0 +1,39 @@
+/**
+ * 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.common.TajoDataTypes.Type;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+
+@Description(
+    functionName = "STDDEV_POP",
+    description = "The population standard deviation of a set of numbers.",
+    example = "> SELECT STDDEV_POP(expr);",
+    returnType = Type.FLOAT8,
+    paramTypes = {@ParamTypes(paramTypes = {Type.INT8})}
+)
+public class StdDevPopLong extends StdDevPop {
+  public StdDevPopLong() {
+    super(new Column[] {
+        new Column("expr", Type.INT8)
+    });
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSamp.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSamp.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSamp.java
new file mode 100644
index 0000000..60f83a4
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSamp.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.engine.function.builtin;
+
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.datum.Datum;
+import org.apache.tajo.datum.DatumFactory;
+import org.apache.tajo.datum.NullDatum;
+import org.apache.tajo.plan.function.FunctionContext;
+
+public abstract class StdDevSamp extends StdDev {
+  public StdDevSamp(Column[] definedArgs) {
+    super(definedArgs);
+  }
+
+  @Override
+  public Datum terminate(FunctionContext ctx) {
+    StdDevContext StdDevCtx = (StdDevContext) ctx;
+    if (StdDevCtx.count <= 1) {
+      return NullDatum.get();
+    }
+
+    return DatumFactory.createFloat8(Math.sqrt(StdDevCtx.squareSumOfDiff / (StdDevCtx.count - 1)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampDouble.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampDouble.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampDouble.java
new file mode 100644
index 0000000..0c06112
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampDouble.java
@@ -0,0 +1,39 @@
+/**
+ * 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.common.TajoDataTypes.Type;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+
+@Description(
+    functionName = "STDDEV_SAMP",
+    description = "The sample standard deviation of a set of numbers.",
+    example = "> SELECT STDDEV_SAMP(expr);",
+    returnType = Type.FLOAT8,
+    paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT8})}
+)
+public class StdDevSampDouble extends StdDevSamp {
+  public StdDevSampDouble() {
+    super(new Column[] {
+        new Column("expr", Type.FLOAT8)
+    });
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampFloat.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampFloat.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampFloat.java
new file mode 100644
index 0000000..5d514f0
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampFloat.java
@@ -0,0 +1,39 @@
+/**
+ * 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.common.TajoDataTypes.Type;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+
+@Description(
+    functionName = "STDDEV_SAMP",
+    description = "The sample standard deviation of a set of numbers.",
+    example = "> SELECT STDDEV_SAMP(expr);",
+    returnType = Type.FLOAT8,
+    paramTypes = {@ParamTypes(paramTypes = {Type.FLOAT4})}
+)
+public class StdDevSampFloat extends StdDevSamp {
+  public StdDevSampFloat() {
+    super(new Column[] {
+        new Column("expr", Type.FLOAT4)
+    });
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampInt.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampInt.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampInt.java
new file mode 100644
index 0000000..8abdaa1
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampInt.java
@@ -0,0 +1,39 @@
+/**
+ * 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.common.TajoDataTypes.Type;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+
+@Description(
+    functionName = "STDDEV_SAMP",
+    description = "The sample standard deviation of a set of numbers.",
+    example = "> SELECT STDDEV_SAMP(expr);",
+    returnType = Type.FLOAT8,
+    paramTypes = {@ParamTypes(paramTypes = {Type.INT4})}
+)
+public class StdDevSampInt extends StdDevSamp {
+  public StdDevSampInt() {
+    super(new Column[] {
+        new Column("expr", Type.INT4)
+    });
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampLong.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampLong.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampLong.java
new file mode 100644
index 0000000..ecbb7a9
--- /dev/null
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/StdDevSampLong.java
@@ -0,0 +1,39 @@
+/**
+ * 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.common.TajoDataTypes.Type;
+import org.apache.tajo.engine.function.annotation.Description;
+import org.apache.tajo.engine.function.annotation.ParamTypes;
+
+@Description(
+    functionName = "STDDEV_SAMP",
+    description = "The sample standard deviation of a set of numbers.",
+    example = "> SELECT STDDEV_SAMP(expr);",
+    returnType = Type.FLOAT8,
+    paramTypes = {@ParamTypes(paramTypes = {Type.INT8})}
+)
+public class StdDevSampLong extends StdDevSamp {
+  public StdDevSampLong() {
+    super(new Column[] {
+        new Column("expr", Type.INT8)
+    });
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/main/proto/InternalTypes.proto
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/proto/InternalTypes.proto b/tajo-core/src/main/proto/InternalTypes.proto
index 1a62bc2..7108991 100644
--- a/tajo-core/src/main/proto/InternalTypes.proto
+++ b/tajo-core/src/main/proto/InternalTypes.proto
@@ -30,3 +30,9 @@ message AvgDoubleProto {
   required double sum = 1;
   required int64 count = 2;
 }
+
+message StdDevProto {
+  required double squareSumOfDiff = 1;
+  required double avg = 2;
+  required int64 count = 3;
+}

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/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 9f68786..bcc5dd6 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
@@ -323,6 +323,135 @@ public class TestBuiltinFunctions extends QueryTestCaseBase {
 
   }
 
+  @Test
+  public void testStdDevSamp() 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 stddev_samp(value_int) as sdsamp_int, stddev_samp(value_long) as sdsamp_long, stddev_samp(value_float) as sdsamp_float, stddev_samp(value_double) as sdsamp_double from table11");
+      String ascExpected = "sdsamp_int,sdsamp_long,sdsamp_float,sdsamp_double\n" +
+          "-------------------------------\n" +
+          "0.7071067811865476,156.97770542341354,1.1313707824635184,72.8319984622144\n";
+
+      assertEquals(ascExpected, resultSetToString(res));
+      res.close();
+    } finally {
+      executeString("DROP TABLE table11 PURGE");
+    }
+
+  }
+
+  @Test
+  public void testStdDevSampWithFewNumbers() 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|-50.5",
+        "2|1|\\N|\\N|\\N",
+        "3|\\N|\\N|\\N|\\N" };
+    TajoTestingCluster.createTable("table11", schema, tableOptions, data, 1);
+
+    try {
+      ResultSet res = executeString("select stddev_samp(value_int) as sdsamp_int, stddev_samp(value_long) as sdsamp_long, stddev_samp(value_float) as sdsamp_float, stddev_samp(value_double) as sdsamp_double from table11");
+      String ascExpected = "sdsamp_int,sdsamp_long,sdsamp_float,sdsamp_double\n" +
+          "-------------------------------\n" +
+          "null,null,null,null\n";
+
+      assertEquals(ascExpected, resultSetToString(res));
+      res.close();
+    } finally {
+      executeString("DROP TABLE table11 PURGE");
+    }
+
+  }
+
+  @Test
+  public void testStdDevPop() 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 stddev_pop(value_int) as sdpop_int, stddev_pop(value_long) as sdpop_long, stddev_pop(value_float) as sdpop_float, stddev_pop(value_double) as sdpop_double from table11");
+      String ascExpected = "sdpop_int,sdpop_long,sdpop_float,sdpop_double\n" +
+          "-------------------------------\n" +
+          "0.5,111.0,0.7999999523162842,51.5\n";
+
+      assertEquals(ascExpected, resultSetToString(res));
+      res.close();
+    } finally {
+      executeString("DROP TABLE table11 PURGE");
+    }
+
+  }
+
+  @Test
+  public void testStdDevPopWithFewNumbers() 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|-50.5",
+        "2|1|\\N|\\N|\\N",
+        "3|\\N|\\N|\\N|\\N" };
+    TajoTestingCluster.createTable("table11", schema, tableOptions, data, 1);
+
+    try {
+      ResultSet res = executeString("select stddev_pop(value_int) as sdpop_int, stddev_pop(value_long) as sdpop_long, stddev_pop(value_float) as sdpop_float, stddev_pop(value_double) as sdpop_double from table11");
+      String ascExpected = "sdpop_int,sdpop_long,sdpop_float,sdpop_double\n" +
+          "-------------------------------\n" +
+          "0.0,null,null,0.0\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/652e4db7/tajo-core/src/test/java/org/apache/tajo/engine/query/TestWindowQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestWindowQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestWindowQuery.java
index 9385db0..9993992 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestWindowQuery.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestWindowQuery.java
@@ -232,35 +232,35 @@ public class TestWindowQuery extends QueryTestCaseBase {
   }
 
   @Test
-  public final void rowNumber1() throws Exception {
+  public final void testRowNumber1() throws Exception {
     ResultSet res = executeQuery();
     assertResultSet(res);
     cleanupQuery(res);
   }
 
   @Test
-  public final void rowNumber2() throws Exception {
+  public final void testRowNumber2() throws Exception {
     ResultSet res = executeQuery();
     assertResultSet(res);
     cleanupQuery(res);
   }
 
   @Test
-  public final void rowNumber3() throws Exception {
+  public final void testRowNumber3() throws Exception {
     ResultSet res = executeQuery();
     assertResultSet(res);
     cleanupQuery(res);
   }
 
   @Test
-  public final void firstValue1() throws Exception {
+  public final void testFirstValue1() throws Exception {
     ResultSet res = executeQuery();
     assertResultSet(res);
     cleanupQuery(res);
   }
 
   @Test
-  public final void firstValueTime() throws Exception {
+  public final void testFirstValueTime() throws Exception {
     KeyValueSet tableOptions = new KeyValueSet();
     tableOptions.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
     tableOptions.set(StorageConstants.TEXT_NULL, "\\\\N");
@@ -288,14 +288,14 @@ public class TestWindowQuery extends QueryTestCaseBase {
   }
 
   @Test
-  public final void lastValue1() throws Exception {
+  public final void testLastValue1() throws Exception {
     ResultSet res = executeQuery();
     assertResultSet(res);
     cleanupQuery(res);
   }
 
   @Test
-  public final void lastValueTime() throws Exception {
+  public final void testLastValueTime() throws Exception {
     KeyValueSet tableOptions = new KeyValueSet();
     tableOptions.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);
     tableOptions.set(StorageConstants.TEXT_NULL, "\\\\N");
@@ -421,6 +421,20 @@ public class TestWindowQuery extends QueryTestCaseBase {
   }
 
   @Test
+  public final void testStdDevSamp1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testStdDevPop1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
   public final void testMultipleWindow() throws Exception {
     KeyValueSet tableOptions = new KeyValueSet();
     tableOptions.set(StorageConstants.TEXT_DELIMITER, StorageConstants.DEFAULT_FIELD_DELIMITER);

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/queries/TestWindowQuery/firstValue1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestWindowQuery/firstValue1.sql b/tajo-core/src/test/resources/queries/TestWindowQuery/firstValue1.sql
deleted file mode 100644
index e9c9c73..0000000
--- a/tajo-core/src/test/resources/queries/TestWindowQuery/firstValue1.sql
+++ /dev/null
@@ -1,18 +0,0 @@
-SELECT
-  l_orderkey,
-  first_value(l_shipmode) over (PARTITION BY L_ORDERKEY order by l_shipmode ) as shipmode_first,
-  first_value(l_linenumber) over (PARTITION BY L_ORDERKEY order by l_linenumber ) as linenumber_first,
-  first_value(l_suppkey_t) over (PARTITION BY L_ORDERKEY order by l_suppkey_t ) as suppkey_first,
-  first_value(l_shipdate_t) over (PARTITION BY L_ORDERKEY order by l_shipdate_t ) as shipdate_first,
-  first_value(l_commitdate_t) over (PARTITION BY L_ORDERKEY order by l_commitdate_t ) as commitdate_first,
-  first_value(l_extendedprice) over (PARTITION BY L_ORDERKEY order by l_extendedprice ) as extendedprice_first,
-  first_value(l_discount_t) over (PARTITION BY L_ORDERKEY order by l_discount_t ) as discount_first
-FROM
-(
-  SELECT
-    l_orderkey,l_partkey,l_suppkey::INT8 as l_suppkey_t,l_linenumber,l_quantity,
-    l_extendedprice,l_discount::FLOAT4 as l_discount_t,l_tax,l_returnflag,l_linestatus,
-    l_shipdate::DATE as l_shipdate_t,l_commitdate::TIMESTAMP as l_commitdate_t,l_receiptdate,l_shipinstruct,l_shipmode,l_comment
-  FROM
-    LINEITEM
-) xx
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/queries/TestWindowQuery/lastValue1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestWindowQuery/lastValue1.sql b/tajo-core/src/test/resources/queries/TestWindowQuery/lastValue1.sql
deleted file mode 100644
index 150b912..0000000
--- a/tajo-core/src/test/resources/queries/TestWindowQuery/lastValue1.sql
+++ /dev/null
@@ -1,18 +0,0 @@
-SELECT
-  l_orderkey,
-  last_value(l_shipmode) over (PARTITION BY L_ORDERKEY order by l_shipmode ) as shipmode_last,
-  last_value(l_linenumber) over (PARTITION BY L_ORDERKEY order by l_linenumber ) as linenumber_last,
-  last_value(l_suppkey_t) over (PARTITION BY L_ORDERKEY order by l_suppkey_t ) as suppkey_last,
-  last_value(l_shipdate_t) over (PARTITION BY L_ORDERKEY order by l_shipdate_t ) as shipdate_last,
-  last_value(l_commitdate_t) over (PARTITION BY L_ORDERKEY order by l_commitdate_t ) as commitdate_last,
-  last_value(l_extendedprice) over (PARTITION BY L_ORDERKEY order by l_extendedprice ) as extendedprice_last,
-  last_value(l_discount_t) over (PARTITION BY L_ORDERKEY order by l_discount_t ) as discount_last
-FROM
-(
-  SELECT
-    l_orderkey,l_partkey,l_suppkey::INT8 as l_suppkey_t,l_linenumber,l_quantity,
-    l_extendedprice,l_discount::FLOAT4 as l_discount_t,l_tax,l_returnflag,l_linestatus,
-    l_shipdate::DATE as l_shipdate_t,l_commitdate::TIMESTAMP as l_commitdate_t,l_receiptdate,l_shipinstruct,l_shipmode,l_comment
-  FROM
-    LINEITEM
-) xx
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/queries/TestWindowQuery/rowNumber1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestWindowQuery/rowNumber1.sql b/tajo-core/src/test/resources/queries/TestWindowQuery/rowNumber1.sql
deleted file mode 100644
index 14a5128..0000000
--- a/tajo-core/src/test/resources/queries/TestWindowQuery/rowNumber1.sql
+++ /dev/null
@@ -1,5 +0,0 @@
-SELECT
-  l_orderkey,
-  row_number() OVER () as row_num
-FROM
-  LINEITEM;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/queries/TestWindowQuery/rowNumber2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestWindowQuery/rowNumber2.sql b/tajo-core/src/test/resources/queries/TestWindowQuery/rowNumber2.sql
deleted file mode 100644
index 2e45120..0000000
--- a/tajo-core/src/test/resources/queries/TestWindowQuery/rowNumber2.sql
+++ /dev/null
@@ -1,5 +0,0 @@
-SELECT
-  l_orderkey,
-  row_number() OVER (PARTITION BY L_ORDERKEY) as row_num
-FROM
-  LINEITEM
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/queries/TestWindowQuery/rowNumber3.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestWindowQuery/rowNumber3.sql b/tajo-core/src/test/resources/queries/TestWindowQuery/rowNumber3.sql
deleted file mode 100644
index 44594c7..0000000
--- a/tajo-core/src/test/resources/queries/TestWindowQuery/rowNumber3.sql
+++ /dev/null
@@ -1,7 +0,0 @@
-SELECT
-  l_orderkey,
-  row_number() OVER (PARTITION BY L_ORDERKEY) as row_num,
-  l_discount,
-  avg(l_discount) OVER (PARTITION BY L_ORDERKEY) as average
-FROM
-  LINEITEM
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/queries/TestWindowQuery/testFirstValue1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestWindowQuery/testFirstValue1.sql b/tajo-core/src/test/resources/queries/TestWindowQuery/testFirstValue1.sql
new file mode 100644
index 0000000..e9c9c73
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestWindowQuery/testFirstValue1.sql
@@ -0,0 +1,18 @@
+SELECT
+  l_orderkey,
+  first_value(l_shipmode) over (PARTITION BY L_ORDERKEY order by l_shipmode ) as shipmode_first,
+  first_value(l_linenumber) over (PARTITION BY L_ORDERKEY order by l_linenumber ) as linenumber_first,
+  first_value(l_suppkey_t) over (PARTITION BY L_ORDERKEY order by l_suppkey_t ) as suppkey_first,
+  first_value(l_shipdate_t) over (PARTITION BY L_ORDERKEY order by l_shipdate_t ) as shipdate_first,
+  first_value(l_commitdate_t) over (PARTITION BY L_ORDERKEY order by l_commitdate_t ) as commitdate_first,
+  first_value(l_extendedprice) over (PARTITION BY L_ORDERKEY order by l_extendedprice ) as extendedprice_first,
+  first_value(l_discount_t) over (PARTITION BY L_ORDERKEY order by l_discount_t ) as discount_first
+FROM
+(
+  SELECT
+    l_orderkey,l_partkey,l_suppkey::INT8 as l_suppkey_t,l_linenumber,l_quantity,
+    l_extendedprice,l_discount::FLOAT4 as l_discount_t,l_tax,l_returnflag,l_linestatus,
+    l_shipdate::DATE as l_shipdate_t,l_commitdate::TIMESTAMP as l_commitdate_t,l_receiptdate,l_shipinstruct,l_shipmode,l_comment
+  FROM
+    LINEITEM
+) xx
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/queries/TestWindowQuery/testLastValue1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestWindowQuery/testLastValue1.sql b/tajo-core/src/test/resources/queries/TestWindowQuery/testLastValue1.sql
new file mode 100644
index 0000000..150b912
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestWindowQuery/testLastValue1.sql
@@ -0,0 +1,18 @@
+SELECT
+  l_orderkey,
+  last_value(l_shipmode) over (PARTITION BY L_ORDERKEY order by l_shipmode ) as shipmode_last,
+  last_value(l_linenumber) over (PARTITION BY L_ORDERKEY order by l_linenumber ) as linenumber_last,
+  last_value(l_suppkey_t) over (PARTITION BY L_ORDERKEY order by l_suppkey_t ) as suppkey_last,
+  last_value(l_shipdate_t) over (PARTITION BY L_ORDERKEY order by l_shipdate_t ) as shipdate_last,
+  last_value(l_commitdate_t) over (PARTITION BY L_ORDERKEY order by l_commitdate_t ) as commitdate_last,
+  last_value(l_extendedprice) over (PARTITION BY L_ORDERKEY order by l_extendedprice ) as extendedprice_last,
+  last_value(l_discount_t) over (PARTITION BY L_ORDERKEY order by l_discount_t ) as discount_last
+FROM
+(
+  SELECT
+    l_orderkey,l_partkey,l_suppkey::INT8 as l_suppkey_t,l_linenumber,l_quantity,
+    l_extendedprice,l_discount::FLOAT4 as l_discount_t,l_tax,l_returnflag,l_linestatus,
+    l_shipdate::DATE as l_shipdate_t,l_commitdate::TIMESTAMP as l_commitdate_t,l_receiptdate,l_shipinstruct,l_shipmode,l_comment
+  FROM
+    LINEITEM
+) xx
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/queries/TestWindowQuery/testRowNumber1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestWindowQuery/testRowNumber1.sql b/tajo-core/src/test/resources/queries/TestWindowQuery/testRowNumber1.sql
new file mode 100644
index 0000000..14a5128
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestWindowQuery/testRowNumber1.sql
@@ -0,0 +1,5 @@
+SELECT
+  l_orderkey,
+  row_number() OVER () as row_num
+FROM
+  LINEITEM;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/queries/TestWindowQuery/testRowNumber2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestWindowQuery/testRowNumber2.sql b/tajo-core/src/test/resources/queries/TestWindowQuery/testRowNumber2.sql
new file mode 100644
index 0000000..2e45120
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestWindowQuery/testRowNumber2.sql
@@ -0,0 +1,5 @@
+SELECT
+  l_orderkey,
+  row_number() OVER (PARTITION BY L_ORDERKEY) as row_num
+FROM
+  LINEITEM
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/queries/TestWindowQuery/testRowNumber3.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestWindowQuery/testRowNumber3.sql b/tajo-core/src/test/resources/queries/TestWindowQuery/testRowNumber3.sql
new file mode 100644
index 0000000..44594c7
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestWindowQuery/testRowNumber3.sql
@@ -0,0 +1,7 @@
+SELECT
+  l_orderkey,
+  row_number() OVER (PARTITION BY L_ORDERKEY) as row_num,
+  l_discount,
+  avg(l_discount) OVER (PARTITION BY L_ORDERKEY) as average
+FROM
+  LINEITEM
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/queries/TestWindowQuery/testStdDevPop1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestWindowQuery/testStdDevPop1.sql b/tajo-core/src/test/resources/queries/TestWindowQuery/testStdDevPop1.sql
new file mode 100644
index 0000000..013c8af
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestWindowQuery/testStdDevPop1.sql
@@ -0,0 +1,14 @@
+SELECT
+  STDDEV_POP(l_linenumber) over (PARTITION BY L_ORDERKEY order by l_shipmode ) as linenumber_stddev_pop,
+  STDDEV_POP(l_suppkey_t) over (PARTITION BY L_ORDERKEY order by l_shipmode ) as suppkey_stddev_pop,
+  STDDEV_POP(l_extendedprice) over (PARTITION BY L_ORDERKEY order by l_shipmode ) as extendedprice_stddev_pop,
+  STDDEV_POP(l_discount_t) over (PARTITION BY L_ORDERKEY order by l_shipmode ) as discount_stddev_pop
+FROM
+(
+  SELECT
+    l_orderkey,l_partkey,l_suppkey::INT8 as l_suppkey_t,l_linenumber,l_quantity,
+    l_extendedprice,l_discount::FLOAT4 as l_discount_t,l_tax,l_returnflag,l_linestatus,
+    l_shipdate::DATE as l_shipdate_t,l_commitdate::TIMESTAMP as l_commitdate_t,l_receiptdate,l_shipinstruct,l_shipmode,l_comment
+  FROM
+    LINEITEM
+) xx
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/queries/TestWindowQuery/testStdDevSamp1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestWindowQuery/testStdDevSamp1.sql b/tajo-core/src/test/resources/queries/TestWindowQuery/testStdDevSamp1.sql
new file mode 100644
index 0000000..4bb5d5c
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestWindowQuery/testStdDevSamp1.sql
@@ -0,0 +1,14 @@
+SELECT
+  STDDEV_SAMP(l_linenumber) over (PARTITION BY L_ORDERKEY order by l_shipmode ) as linenumber_stddev_samp,
+  STDDEV_SAMP(l_suppkey_t) over (PARTITION BY L_ORDERKEY order by l_shipmode ) as suppkey_stddev_samp,
+  STDDEV_SAMP(l_extendedprice) over (PARTITION BY L_ORDERKEY order by l_shipmode ) as extendedprice_stddev_samp,
+  STDDEV_SAMP(l_discount_t) over (PARTITION BY L_ORDERKEY order by l_shipmode ) as discount_stddev_samp
+FROM
+(
+  SELECT
+    l_orderkey,l_partkey,l_suppkey::INT8 as l_suppkey_t,l_linenumber,l_quantity,
+    l_extendedprice,l_discount::FLOAT4 as l_discount_t,l_tax,l_returnflag,l_linestatus,
+    l_shipdate::DATE as l_shipdate_t,l_commitdate::TIMESTAMP as l_commitdate_t,l_receiptdate,l_shipinstruct,l_shipmode,l_comment
+  FROM
+    LINEITEM
+) xx
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/results/TestWindowQuery/firstValue1.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestWindowQuery/firstValue1.result b/tajo-core/src/test/resources/results/TestWindowQuery/firstValue1.result
deleted file mode 100644
index 8ca4ea9..0000000
--- a/tajo-core/src/test/resources/results/TestWindowQuery/firstValue1.result
+++ /dev/null
@@ -1,7 +0,0 @@
-l_orderkey,shipmode_first,linenumber_first,suppkey_first,shipdate_first,commitdate_first,extendedprice_first,discount_first
--------------------------------
-1,MAIL,1,7311,1996-03-13,1996-02-12 00:00:00,21168.23,0.04
-1,MAIL,1,7311,1996-03-13,1996-02-12 00:00:00,21168.23,0.04
-2,RAIL,1,1191,1997-01-28,1997-01-14 00:00:00,44694.46,0.0
-3,AIR,1,1798,1993-11-09,1993-12-20 00:00:00,46796.47,0.06
-3,AIR,1,1798,1993-11-09,1993-12-20 00:00:00,46796.47,0.06
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/results/TestWindowQuery/lastValue1.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestWindowQuery/lastValue1.result b/tajo-core/src/test/resources/results/TestWindowQuery/lastValue1.result
deleted file mode 100644
index 825f982..0000000
--- a/tajo-core/src/test/resources/results/TestWindowQuery/lastValue1.result
+++ /dev/null
@@ -1,7 +0,0 @@
-l_orderkey,shipmode_last,linenumber_last,suppkey_last,shipdate_last,commitdate_last,extendedprice_last,discount_last
--------------------------------
-1,TRUCK,2,7706,1996-04-12,1996-02-28 00:00:00,45983.16,0.09
-1,TRUCK,2,7706,1996-04-12,1996-02-28 00:00:00,45983.16,0.09
-2,RAIL,1,1191,1997-01-28,1997-01-14 00:00:00,44694.46,0.0
-3,RAIL,2,6540,1994-02-02,1994-01-04 00:00:00,54058.05,0.1
-3,RAIL,2,6540,1994-02-02,1994-01-04 00:00:00,54058.05,0.1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/results/TestWindowQuery/rowNumber1.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestWindowQuery/rowNumber1.result b/tajo-core/src/test/resources/results/TestWindowQuery/rowNumber1.result
deleted file mode 100644
index 5fc49ee..0000000
--- a/tajo-core/src/test/resources/results/TestWindowQuery/rowNumber1.result
+++ /dev/null
@@ -1,7 +0,0 @@
-l_orderkey,row_num
--------------------------------
-1,1
-1,2
-2,3
-3,4
-3,5
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/results/TestWindowQuery/rowNumber2.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestWindowQuery/rowNumber2.result b/tajo-core/src/test/resources/results/TestWindowQuery/rowNumber2.result
deleted file mode 100644
index db02a76..0000000
--- a/tajo-core/src/test/resources/results/TestWindowQuery/rowNumber2.result
+++ /dev/null
@@ -1,7 +0,0 @@
-l_orderkey,row_num
--------------------------------
-1,1
-1,2
-2,1
-3,1
-3,2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/results/TestWindowQuery/rowNumber3.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestWindowQuery/rowNumber3.result b/tajo-core/src/test/resources/results/TestWindowQuery/rowNumber3.result
deleted file mode 100644
index 1d780ff..0000000
--- a/tajo-core/src/test/resources/results/TestWindowQuery/rowNumber3.result
+++ /dev/null
@@ -1,7 +0,0 @@
-l_orderkey,row_num,l_discount,average
--------------------------------
-1,1,0.04,0.065
-1,2,0.09,0.065
-2,1,0.0,0.0
-3,1,0.06,0.08
-3,2,0.1,0.08
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/results/TestWindowQuery/testFirstValue1.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestWindowQuery/testFirstValue1.result b/tajo-core/src/test/resources/results/TestWindowQuery/testFirstValue1.result
new file mode 100644
index 0000000..8ca4ea9
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestWindowQuery/testFirstValue1.result
@@ -0,0 +1,7 @@
+l_orderkey,shipmode_first,linenumber_first,suppkey_first,shipdate_first,commitdate_first,extendedprice_first,discount_first
+-------------------------------
+1,MAIL,1,7311,1996-03-13,1996-02-12 00:00:00,21168.23,0.04
+1,MAIL,1,7311,1996-03-13,1996-02-12 00:00:00,21168.23,0.04
+2,RAIL,1,1191,1997-01-28,1997-01-14 00:00:00,44694.46,0.0
+3,AIR,1,1798,1993-11-09,1993-12-20 00:00:00,46796.47,0.06
+3,AIR,1,1798,1993-11-09,1993-12-20 00:00:00,46796.47,0.06
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/results/TestWindowQuery/testLastValue1.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestWindowQuery/testLastValue1.result b/tajo-core/src/test/resources/results/TestWindowQuery/testLastValue1.result
new file mode 100644
index 0000000..825f982
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestWindowQuery/testLastValue1.result
@@ -0,0 +1,7 @@
+l_orderkey,shipmode_last,linenumber_last,suppkey_last,shipdate_last,commitdate_last,extendedprice_last,discount_last
+-------------------------------
+1,TRUCK,2,7706,1996-04-12,1996-02-28 00:00:00,45983.16,0.09
+1,TRUCK,2,7706,1996-04-12,1996-02-28 00:00:00,45983.16,0.09
+2,RAIL,1,1191,1997-01-28,1997-01-14 00:00:00,44694.46,0.0
+3,RAIL,2,6540,1994-02-02,1994-01-04 00:00:00,54058.05,0.1
+3,RAIL,2,6540,1994-02-02,1994-01-04 00:00:00,54058.05,0.1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/results/TestWindowQuery/testRowNumber1.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestWindowQuery/testRowNumber1.result b/tajo-core/src/test/resources/results/TestWindowQuery/testRowNumber1.result
new file mode 100644
index 0000000..5fc49ee
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestWindowQuery/testRowNumber1.result
@@ -0,0 +1,7 @@
+l_orderkey,row_num
+-------------------------------
+1,1
+1,2
+2,3
+3,4
+3,5
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/results/TestWindowQuery/testRowNumber2.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestWindowQuery/testRowNumber2.result b/tajo-core/src/test/resources/results/TestWindowQuery/testRowNumber2.result
new file mode 100644
index 0000000..db02a76
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestWindowQuery/testRowNumber2.result
@@ -0,0 +1,7 @@
+l_orderkey,row_num
+-------------------------------
+1,1
+1,2
+2,1
+3,1
+3,2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/results/TestWindowQuery/testRowNumber3.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestWindowQuery/testRowNumber3.result b/tajo-core/src/test/resources/results/TestWindowQuery/testRowNumber3.result
new file mode 100644
index 0000000..1d780ff
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestWindowQuery/testRowNumber3.result
@@ -0,0 +1,7 @@
+l_orderkey,row_num,l_discount,average
+-------------------------------
+1,1,0.04,0.065
+1,2,0.09,0.065
+2,1,0.0,0.0
+3,1,0.06,0.08
+3,2,0.1,0.08
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/results/TestWindowQuery/testStdDevPop1.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestWindowQuery/testStdDevPop1.result b/tajo-core/src/test/resources/results/TestWindowQuery/testStdDevPop1.result
new file mode 100644
index 0000000..ef2e5d3
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestWindowQuery/testStdDevPop1.result
@@ -0,0 +1,7 @@
+linenumber_stddev_pop,suppkey_stddev_pop,extendedprice_stddev_pop,discount_stddev_pop
+-------------------------------
+0.5,197.5,12407.465,0.02500000223517418
+0.5,197.5,12407.465,0.02500000223517418
+0.0,0.0,0.0,0.0
+0.5,2371.0,3630.790000000001,0.02000000141561031
+0.5,2371.0,3630.790000000001,0.02000000141561031
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tajo/blob/652e4db7/tajo-core/src/test/resources/results/TestWindowQuery/testStdDevSamp1.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestWindowQuery/testStdDevSamp1.result b/tajo-core/src/test/resources/results/TestWindowQuery/testStdDevSamp1.result
new file mode 100644
index 0000000..5db6227
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestWindowQuery/testStdDevSamp1.result
@@ -0,0 +1,7 @@
+linenumber_stddev_samp,suppkey_stddev_samp,extendedprice_stddev_samp,discount_stddev_samp
+-------------------------------
+0.7071067811865476,279.30717856868625,17546.805277669493,0.03535534222034101
+0.7071067811865476,279.30717856868625,17546.805277669493,0.03535534222034101
+null,null,null,null
+0.7071067811865476,3353.1003563866084,5134.7124601286105,0.0282842732494372
+0.7071067811865476,3353.1003563866084,5134.7124601286105,0.0282842732494372
\ No newline at end of file