You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hivemall.apache.org by my...@apache.org on 2016/12/19 05:05:19 UTC

incubator-hivemall git commit: Close #11: Implemented SST-based change point detector

Repository: incubator-hivemall
Updated Branches:
  refs/heads/master 7a3193a8b -> 9c0aae3d4


Close #11: Implemented SST-based change point detector


Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/9c0aae3d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/9c0aae3d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/9c0aae3d

Branch: refs/heads/master
Commit: 9c0aae3d4ef9b9c3fecac1f238f70624b493669c
Parents: 7a3193a
Author: Takuya Kitazawa <k....@gmail.com>
Authored: Mon Dec 19 14:04:53 2016 +0900
Committer: myui <yu...@gmail.com>
Committed: Mon Dec 19 14:04:53 2016 +0900

----------------------------------------------------------------------
 .../java/hivemall/anomaly/ChangeFinderUDF.java  |   2 +
 .../anomaly/SingularSpectrumTransform.java      | 201 +++++++++++++++
 .../anomaly/SingularSpectrumTransformUDF.java   | 253 +++++++++++++++++++
 .../java/hivemall/utils/lang/Preconditions.java |  42 +++
 .../java/hivemall/utils/math/MathUtils.java     |   9 +
 .../java/hivemall/utils/math/MatrixUtils.java   | 213 +++++++++++++++-
 .../anomaly/SingularSpectrumTransformTest.java  | 147 +++++++++++
 .../hivemall/utils/lang/PreconditionsTest.java  |  29 ++-
 .../hivemall/utils/math/MatrixUtilsTest.java    |  67 +++++
 resources/eclipse-style.xml                     |   2 +-
 10 files changed, 955 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/9c0aae3d/core/src/main/java/hivemall/anomaly/ChangeFinderUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/anomaly/ChangeFinderUDF.java b/core/src/main/java/hivemall/anomaly/ChangeFinderUDF.java
index 7498255..e942527 100644
--- a/core/src/main/java/hivemall/anomaly/ChangeFinderUDF.java
+++ b/core/src/main/java/hivemall/anomaly/ChangeFinderUDF.java
@@ -36,6 +36,7 @@ import org.apache.commons.cli.Options;
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.UDFType;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -49,6 +50,7 @@ import org.apache.hadoop.io.BooleanWritable;
         value = "_FUNC_(double|array<double> x [, const string options])"
                 + " - Returns outlier/change-point scores and decisions using ChangeFinder."
                 + " It will return a tuple <double outlier_score, double changepoint_score [, boolean is_anomaly [, boolean is_changepoint]]")
+@UDFType(deterministic = false, stateful = true)
 public final class ChangeFinderUDF extends UDFWithOptions {
 
     private transient Parameters _params;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/9c0aae3d/core/src/main/java/hivemall/anomaly/SingularSpectrumTransform.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/anomaly/SingularSpectrumTransform.java b/core/src/main/java/hivemall/anomaly/SingularSpectrumTransform.java
new file mode 100644
index 0000000..ba1c9c0
--- /dev/null
+++ b/core/src/main/java/hivemall/anomaly/SingularSpectrumTransform.java
@@ -0,0 +1,201 @@
+/*
+ * 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 hivemall.anomaly;
+
+import hivemall.anomaly.SingularSpectrumTransformUDF.Parameters;
+import hivemall.anomaly.SingularSpectrumTransformUDF.ScoreFunction;
+import hivemall.anomaly.SingularSpectrumTransformUDF.SingularSpectrumTransformInterface;
+import hivemall.utils.collections.DoubleRingBuffer;
+import hivemall.utils.lang.Preconditions;
+import hivemall.utils.math.MatrixUtils;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.TreeMap;
+
+import javax.annotation.Nonnull;
+
+import org.apache.commons.math3.linear.Array2DRowRealMatrix;
+import org.apache.commons.math3.linear.RealMatrix;
+import org.apache.commons.math3.linear.SingularValueDecomposition;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+
+final class SingularSpectrumTransform implements SingularSpectrumTransformInterface {
+
+    @Nonnull
+    private final PrimitiveObjectInspector oi;
+    @Nonnull
+    private final ScoreFunction scoreFunc;
+
+    @Nonnull
+    private final int window;
+    @Nonnull
+    private final int nPastWindow;
+    @Nonnull
+    private final int nCurrentWindow;
+    @Nonnull
+    private final int pastSize;
+    @Nonnull
+    private final int currentSize;
+    @Nonnull
+    private final int currentOffset;
+    @Nonnull
+    private final int r;
+    @Nonnull
+    private final int k;
+
+    @Nonnull
+    private final DoubleRingBuffer xRing;
+    @Nonnull
+    private final double[] xSeries;
+
+    @Nonnull
+    private final double[] q;
+
+    SingularSpectrumTransform(@Nonnull Parameters params, @Nonnull PrimitiveObjectInspector oi) {
+        this.oi = oi;
+        this.scoreFunc = params.scoreFunc;
+
+        this.window = params.w;
+        this.nPastWindow = params.n;
+        this.nCurrentWindow = params.m;
+        this.pastSize = window + nPastWindow;
+        this.currentSize = window + nCurrentWindow;
+        this.currentOffset = params.g;
+        this.r = params.r;
+        this.k = params.k;
+        Preconditions.checkArgument(params.k >= params.r);
+
+        // (w + n) past samples for the n-past-windows
+        // (w + m) current samples for the m-current-windows, starting from offset g
+        // => need to hold past (w + n + g + w + m) samples from the latest sample
+        int holdSampleSize = pastSize + currentOffset + currentSize;
+
+        this.xRing = new DoubleRingBuffer(holdSampleSize);
+        this.xSeries = new double[holdSampleSize];
+
+        this.q = new double[window];
+        double norm = 0.d;
+        for (int i = 0; i < window; i++) {
+            this.q[i] = Math.random();
+            norm += q[i] * q[i];
+        }
+        norm = Math.sqrt(norm);
+        // normalize
+        for (int i = 0; i < window; i++) {
+            this.q[i] = q[i] / norm;
+        }
+    }
+
+    @Override
+    public void update(@Nonnull final Object arg, @Nonnull final double[] outScores)
+            throws HiveException {
+        double x = PrimitiveObjectInspectorUtils.getDouble(arg, oi);
+        xRing.add(x).toArray(xSeries, true /* FIFO */);
+
+        // need to wait until the buffer is filled
+        if (!xRing.isFull()) {
+            outScores[0] = 0.d;
+        } else {
+            // create past trajectory matrix and find its left singular vectors
+            RealMatrix H = new Array2DRowRealMatrix(window, nPastWindow);
+            for (int i = 0; i < nPastWindow; i++) {
+                H.setColumn(i, Arrays.copyOfRange(xSeries, i, i + window));
+            }
+
+            // create current trajectory matrix and find its left singular vectors
+            RealMatrix G = new Array2DRowRealMatrix(window, nCurrentWindow);
+            int currentHead = pastSize + currentOffset;
+            for (int i = 0; i < nCurrentWindow; i++) {
+                G.setColumn(i,
+                    Arrays.copyOfRange(xSeries, currentHead + i, currentHead + i + window));
+            }
+
+            switch (scoreFunc) {
+                case svd:
+                    outScores[0] = computeScoreSVD(H, G);
+                    break;
+                case ika:
+                    outScores[0] = computeScoreIKA(H, G);
+                    break;
+                default:
+                    throw new IllegalStateException("Unexpected score function: " + scoreFunc);
+            }
+        }
+    }
+
+    /**
+     * Singular Value Decomposition (SVD) based naive scoring.
+     */
+    private double computeScoreSVD(@Nonnull final RealMatrix H, @Nonnull final RealMatrix G) {
+        SingularValueDecomposition svdH = new SingularValueDecomposition(H);
+        RealMatrix UT = svdH.getUT();
+
+        SingularValueDecomposition svdG = new SingularValueDecomposition(G);
+        RealMatrix Q = svdG.getU();
+
+        // find the largest singular value for the r principal components
+        RealMatrix UTQ = UT.getSubMatrix(0, r - 1, 0, window - 1).multiply(
+            Q.getSubMatrix(0, window - 1, 0, r - 1));
+        SingularValueDecomposition svdUTQ = new SingularValueDecomposition(UTQ);
+        double[] s = svdUTQ.getSingularValues();
+
+        return 1.d - s[0];
+    }
+
+    /**
+     * Implicit Krylov Approximation (IKA) based naive scoring.
+     *
+     * Number of iterations for the Power method and QR method is fixed to 1 for efficiency. This
+     * may cause failure (i.e. meaningless scores) depending on datasets and initial values.
+     *
+     */
+    private double computeScoreIKA(@Nonnull final RealMatrix H, @Nonnull final RealMatrix G) {
+        // assuming n = m = window, and keep track the left singular vector as `q`
+        MatrixUtils.power1(G, q, 1, q, new double[window]);
+
+        RealMatrix T = new Array2DRowRealMatrix(k, k);
+        MatrixUtils.lanczosTridiagonalization(H.multiply(H.transpose()), q, T);
+
+        double[] eigvals = new double[k];
+        RealMatrix eigvecs = new Array2DRowRealMatrix(k, k);
+        MatrixUtils.tridiagonalEigen(T, 1, eigvals, eigvecs);
+
+        // tridiagonalEigen() returns unordered eigenvalues,
+        // so the top-r eigenvectors should be picked carefully
+        TreeMap<Double, Integer> map = new TreeMap<Double, Integer>(Collections.reverseOrder());
+        for (int i = 0; i < k; i++) {
+            map.put(eigvals[i], i);
+        }
+        Iterator<Integer> indicies = map.values().iterator();
+
+        double s = 0.d;
+        for (int i = 0; i < r; i++) {
+            if(!indicies.hasNext()) {
+                throw new IllegalStateException("Should not happen");
+            }
+            double v = eigvecs.getEntry(0, indicies.next().intValue());
+            s += v * v;
+        }
+        return 1.d - Math.sqrt(s);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/9c0aae3d/core/src/main/java/hivemall/anomaly/SingularSpectrumTransformUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/anomaly/SingularSpectrumTransformUDF.java b/core/src/main/java/hivemall/anomaly/SingularSpectrumTransformUDF.java
new file mode 100644
index 0000000..d699a95
--- /dev/null
+++ b/core/src/main/java/hivemall/anomaly/SingularSpectrumTransformUDF.java
@@ -0,0 +1,253 @@
+/*
+ * 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 hivemall.anomaly;
+
+import hivemall.UDFWithOptions;
+import hivemall.utils.hadoop.HiveUtils;
+import hivemall.utils.lang.Preconditions;
+import hivemall.utils.lang.Primitives;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.UDFType;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.BooleanWritable;
+
+/**
+ * Change-point detection based on Singular Spectrum Transformation (SST).
+ *
+ * References:
+ * <ul>
+ * <li>T. Ide and K. Inoue, "Knowledge Discovery from Heterogeneous Dynamic Systems using Change-Point Correlations", SDM'05.</li>
+ * <li>T. Ide and K. Tsuda, "Change-point detection using Krylov subspace learning", SDM'07.</li>
+ * </ul>
+ */
+@Description(
+        name = "sst",
+        value = "_FUNC_(double|array<double> x [, const string options])"
+                + " - Returns change-point scores and decisions using Singular Spectrum Transformation (SST)."
+                + " It will return a tuple <double changepoint_score [, boolean is_changepoint]>")
+@UDFType(deterministic = false, stateful = true)
+public final class SingularSpectrumTransformUDF extends UDFWithOptions {
+
+    private transient Parameters _params;
+    private transient SingularSpectrumTransform _sst;
+
+    private transient double[] _scores;
+    private transient Object[] _result;
+    private transient DoubleWritable _changepointScore;
+    @Nullable
+    private transient BooleanWritable _isChangepoint;
+
+    public SingularSpectrumTransformUDF() {}
+
+    // Visible for testing
+    Parameters getParameters() {
+        return _params;
+    }
+
+    @Override
+    protected Options getOptions() {
+        Options opts = new Options();
+        opts.addOption("w", "window", true,
+            "Number of samples which affects change-point score [default: 30]");
+        opts.addOption("n", "n_past", true,
+            "Number of past windows for change-point scoring [default: equal to `w` = 30]");
+        opts.addOption("m", "n_current", true,
+            "Number of current windows for change-point scoring [default: equal to `w` = 30]");
+        opts.addOption("g", "current_offset", true,
+            "Offset of the current windows from the updating sample [default: `-w` = -30]");
+        opts.addOption("r", "n_component", true,
+            "Number of singular vectors (i.e. principal components) [default: 3]");
+        opts.addOption(
+            "k",
+            "n_dim",
+            true,
+            "Number of dimensions for the Krylov subspaces [default: 5 (`2*r` if `r` is even, `2*r-1` otherwise)]");
+        opts.addOption("score", "scorefunc", true, "Score function [default: svd, ika]");
+        opts.addOption(
+            "th",
+            "threshold",
+            true,
+            "Score threshold (inclusive) for determining change-point existence [default: -1, do not output decision]");
+        return opts;
+    }
+
+    @Override
+    protected CommandLine processOptions(String optionValues) throws UDFArgumentException {
+        CommandLine cl = parseOptions(optionValues);
+
+        this._params.w = Primitives.parseInt(cl.getOptionValue("w"), _params.w);
+        this._params.n = Primitives.parseInt(cl.getOptionValue("n"), _params.w);
+        this._params.m = Primitives.parseInt(cl.getOptionValue("m"), _params.w);
+        this._params.g = Primitives.parseInt(cl.getOptionValue("g"), -1 * _params.w);
+        this._params.r = Primitives.parseInt(cl.getOptionValue("r"), _params.r);
+        this._params.k = Primitives.parseInt(cl.getOptionValue("k"),
+            (_params.r % 2 == 0) ? (2 * _params.r) : (2 * _params.r - 1));
+
+        this._params.scoreFunc = ScoreFunction.resolve(cl.getOptionValue("scorefunc",
+            ScoreFunction.svd.name()));
+        if ((_params.w != _params.n || _params.w != _params.m)
+                && _params.scoreFunc == ScoreFunction.ika) {
+            throw new UDFArgumentException("IKA-based efficient SST requires w = n = m");
+        }
+
+        this._params.changepointThreshold = Primitives.parseDouble(cl.getOptionValue("th"),
+            _params.changepointThreshold);
+
+        Preconditions.checkArgument(_params.w >= 2, "w must be greather than 1: " + _params.w,
+            UDFArgumentException.class);
+        Preconditions.checkArgument(_params.r >= 1, "r must be greater than 0: " + _params.r,
+            UDFArgumentException.class);
+        Preconditions.checkArgument(_params.k >= 1, "k must be greater than 0: " + _params.k,
+            UDFArgumentException.class);
+        Preconditions.checkArgument(_params.k >= _params.r,
+            "k must be equals to or greather than r: k=" + _params.k + ", r" + _params.r,
+            UDFArgumentException.class);
+        Preconditions.checkArgument(_params.changepointThreshold > 0.d
+                && _params.changepointThreshold < 1.d,
+            "changepointThreshold must be in range (0, 1): " + _params.changepointThreshold,
+            UDFArgumentException.class);
+
+        return cl;
+    }
+
+    @Override
+    public ObjectInspector initialize(@Nonnull ObjectInspector[] argOIs)
+            throws UDFArgumentException {
+        if (argOIs.length < 1 || argOIs.length > 2) {
+            throw new UDFArgumentException(
+                "_FUNC_(double|array<double> x [, const string options]) takes 1 or 2 arguments: "
+                        + Arrays.toString(argOIs));
+        }
+
+        this._params = new Parameters();
+        if (argOIs.length == 2) {
+            String options = HiveUtils.getConstString(argOIs[1]);
+            processOptions(options);
+        }
+
+        ObjectInspector argOI0 = argOIs[0];
+        PrimitiveObjectInspector xOI = HiveUtils.asDoubleCompatibleOI(argOI0);
+        this._sst = new SingularSpectrumTransform(_params, xOI);
+
+        this._scores = new double[1];
+
+        final Object[] result;
+        final ArrayList<String> fieldNames = new ArrayList<String>();
+        final ArrayList<ObjectInspector> fieldOIs = new ArrayList<ObjectInspector>();
+        fieldNames.add("changepoint_score");
+        fieldOIs.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+        if (_params.changepointThreshold != -1d) {
+            fieldNames.add("is_changepoint");
+            fieldOIs.add(PrimitiveObjectInspectorFactory.writableBooleanObjectInspector);
+            result = new Object[2];
+            this._isChangepoint = new BooleanWritable(false);
+            result[1] = _isChangepoint;
+        } else {
+            result = new Object[1];
+        }
+        this._changepointScore = new DoubleWritable(0.d);
+        result[0] = _changepointScore;
+        this._result = result;
+
+        return ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames, fieldOIs);
+    }
+
+    @Override
+    public Object[] evaluate(@Nonnull DeferredObject[] args) throws HiveException {
+        Object x = args[0].get();
+        if (x == null) {
+            return _result;
+        }
+
+        _sst.update(x, _scores);
+
+        double changepointScore = _scores[0];
+        _changepointScore.set(changepointScore);
+        if (_isChangepoint != null) {
+            _isChangepoint.set(changepointScore >= _params.changepointThreshold);
+        }
+
+        return _result;
+    }
+
+    @Override
+    public void close() throws IOException {
+        this._result = null;
+        this._changepointScore = null;
+        this._isChangepoint = null;
+    }
+
+    @Override
+    public String getDisplayString(String[] children) {
+        return "sst(" + Arrays.toString(children) + ")";
+    }
+
+    static final class Parameters {
+        int w = 30;
+        int n = 30;
+        int m = 30;
+        int g = -30;
+        int r = 3;
+        int k = 5;
+        @Nonnull
+        ScoreFunction scoreFunc = ScoreFunction.svd;
+        double changepointThreshold = -1.d;
+
+        Parameters() {}
+
+        void set(@Nonnull ScoreFunction func) {
+            this.scoreFunc = func;
+        }
+    }
+
+    public interface SingularSpectrumTransformInterface {
+        void update(@Nonnull Object arg, @Nonnull double[] outScores) throws HiveException;
+    }
+
+    public enum ScoreFunction {
+        svd, ika;
+
+        static ScoreFunction resolve(@Nullable final String name) throws UDFArgumentException {
+            if (svd.name().equalsIgnoreCase(name)) {
+                return svd;
+            } else if (ika.name().equalsIgnoreCase(name)) {
+                return ika;
+            } else {
+                throw new UDFArgumentException("Unsupported ScoreFunction: " + name);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/9c0aae3d/core/src/main/java/hivemall/utils/lang/Preconditions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/Preconditions.java b/core/src/main/java/hivemall/utils/lang/Preconditions.java
index 9f76bd6..af63127 100644
--- a/core/src/main/java/hivemall/utils/lang/Preconditions.java
+++ b/core/src/main/java/hivemall/utils/lang/Preconditions.java
@@ -18,6 +18,9 @@
  */
 package hivemall.utils.lang;
 
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
@@ -47,6 +50,26 @@ public final class Preconditions {
         return reference;
     }
 
+    public static <T, E extends Throwable> T checkNotNull(@Nullable T reference,
+            @Nonnull String errorMessage, @Nonnull Class<E> clazz) throws E {
+        if (reference == null) {
+            final E throwable;
+            try {
+                Constructor<E> constructor = clazz.getConstructor(String.class);
+                throwable = constructor.newInstance(errorMessage);
+            } catch (NoSuchMethodException | SecurityException e1) {
+                throw new IllegalStateException("Failed to get a Constructor(String): "
+                        + clazz.getName(), e1);
+            } catch (InstantiationException | IllegalAccessException | IllegalArgumentException
+                    | InvocationTargetException e2) {
+                throw new IllegalStateException(
+                    "Failed to instantiate a class: " + clazz.getName(), e2);
+            }
+            throw throwable;
+        }
+        return reference;
+    }
+
     public static <T> T checkNotNull(T reference, @Nullable Object errorMessage) {
         if (reference == null) {
             throw new NullPointerException(String.valueOf(errorMessage));
@@ -74,6 +97,25 @@ public final class Preconditions {
         }
     }
 
+    public static <E extends Throwable> void checkArgument(boolean expression,
+            @Nonnull String errorMessage, @Nonnull Class<E> clazz) throws E {
+        if (!expression) {
+            final E throwable;
+            try {
+                Constructor<E> constructor = clazz.getConstructor(String.class);
+                throwable = constructor.newInstance(errorMessage);
+            } catch (NoSuchMethodException | SecurityException e1) {
+                throw new IllegalStateException("Failed to get a Constructor(String): "
+                        + clazz.getName(), e1);
+            } catch (InstantiationException | IllegalAccessException | IllegalArgumentException
+                    | InvocationTargetException e2) {
+                throw new IllegalStateException(
+                    "Failed to instantiate a class: " + clazz.getName(), e2);
+            }
+            throw throwable;
+        }
+    }
+
     public static void checkArgument(boolean expression, @Nullable Object errorMessage) {
         if (!expression) {
             throw new IllegalArgumentException(String.valueOf(errorMessage));

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/9c0aae3d/core/src/main/java/hivemall/utils/math/MathUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/math/MathUtils.java b/core/src/main/java/hivemall/utils/math/MathUtils.java
index 75ace9c..ae9f029 100644
--- a/core/src/main/java/hivemall/utils/math/MathUtils.java
+++ b/core/src/main/java/hivemall/utils/math/MathUtils.java
@@ -292,4 +292,13 @@ public final class MathUtils {
         return true;
     }
 
+    public static double sign(final double x) {
+        if (x < 0.d) {
+            return -1.d;
+        } else if (x > 0.d) {
+            return 1.d;
+        }
+        return 0; // 0 or NaN
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/9c0aae3d/core/src/main/java/hivemall/utils/math/MatrixUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/math/MatrixUtils.java b/core/src/main/java/hivemall/utils/math/MatrixUtils.java
index 9659526..66d6e8c 100644
--- a/core/src/main/java/hivemall/utils/math/MatrixUtils.java
+++ b/core/src/main/java/hivemall/utils/math/MatrixUtils.java
@@ -21,14 +21,19 @@ package hivemall.utils.math;
 import hivemall.utils.collections.DoubleArrayList;
 import hivemall.utils.lang.Preconditions;
 
+import java.util.Arrays;
+
 import javax.annotation.Nonnull;
 
+import org.apache.commons.math3.linear.Array2DRowRealMatrix;
+import org.apache.commons.math3.linear.ArrayRealVector;
 import org.apache.commons.math3.linear.BlockRealMatrix;
 import org.apache.commons.math3.linear.DecompositionSolver;
 import org.apache.commons.math3.linear.DefaultRealMatrixPreservingVisitor;
 import org.apache.commons.math3.linear.LUDecomposition;
 import org.apache.commons.math3.linear.RealMatrix;
 import org.apache.commons.math3.linear.RealMatrixPreservingVisitor;
+import org.apache.commons.math3.linear.RealVector;
 import org.apache.commons.math3.linear.SingularMatrixException;
 import org.apache.commons.math3.linear.SingularValueDecomposition;
 
@@ -39,12 +44,11 @@ public final class MatrixUtils {
     /**
      * Solve Yule-walker equation by Levinson-Durbin Recursion.
      * 
+     * <pre>
      * R_j = \u2211_{i=1}^{k} A_i R_{j-i} where j = 1..k, R_{-i} = R'_i
+     * </pre>
      * 
-     * cf.
-     * http://www.emptyloop.com/technotes/a%20tutorial%20on%20linear%20prediction%20and%20levinson
-     * -durbin.pdf
-     * 
+     * @see http://www.emptyloop.com/technotes/a%20tutorial%20on%20linear%20prediction%20and%20levinson-durbin.pdf
      * @param R autocovariance where |R| >= order
      * @param A coefficient to be solved where |A| >= order + 1
      * @return E variance of prediction error
@@ -134,8 +138,7 @@ public final class MatrixUtils {
     /**
      * Fit an AR(order) model using the Burg's method.
      *
-     * cf. https://searchcode.com/codesearch/view/9503568/
-     *
+     * @see https://searchcode.com/codesearch/view/9503568/
      * @param X data vector to estimate where |X| >= order
      * @param A coefficient to be solved where |A| >= order + 1
      * @return E variance of white noise
@@ -456,14 +459,24 @@ public final class MatrixUtils {
         return inv;
     }
 
-
-    @Nonnull
     public static double det(@Nonnull final RealMatrix m) {
         LUDecomposition LU = new LUDecomposition(m);
         return LU.getDeterminant();
     }
 
     /**
+     * Return a 2-D array with ones on the diagonal and zeros elsewhere.
+     */
+    @Nonnull
+    public static double[][] eye(int n) {
+        final double[][] eye = new double[n][n];
+        for (int i = 0; i < n; i++) {
+            eye[i][i] = 1;
+        }
+        return eye;
+    }
+
+    /**
      * L = A x R
      * 
      * @return a matrix A that minimizes A x R - L
@@ -494,4 +507,188 @@ public final class MatrixUtils {
         return A;
     }
 
+    /**
+     * Find the first singular vector/value of a matrix A based on the Power method.
+     * 
+     * @see http://www.cs.yale.edu/homes/el327/datamining2013aFiles/07_singular_value_decomposition.pdf
+     * @param A target matrix
+     * @param x0 initial vector
+     * @param nIter number of iterations for the Power method
+     * @param u 1st left singular vector
+     * @param v 1st right singular vector
+     * @return 1st singular value
+     */
+    public static double power1(@Nonnull final RealMatrix A, @Nonnull final double[] x0,
+            final int nIter, @Nonnull final double[] u, @Nonnull final double[] v) {
+        Preconditions.checkArgument(A.getColumnDimension() == x0.length,
+            "Column size of A and length of x should be same");
+        Preconditions.checkArgument(A.getRowDimension() == u.length,
+            "Row size of A and length of u should be same");
+        Preconditions.checkArgument(x0.length == v.length, "Length of x and u should be same");
+        Preconditions.checkArgument(nIter >= 1, "Invalid number of iterations: " + nIter);
+
+        RealMatrix AtA = A.transpose().multiply(A);
+
+        RealVector x = new ArrayRealVector(x0);
+        for (int i = 0; i < nIter; i++) {
+            x = AtA.operate(x);
+        }
+
+        double xNorm = x.getNorm();
+        for (int i = 0, n = v.length; i < n; i++) {
+            v[i] = x.getEntry(i) / xNorm;
+        }
+
+        RealVector Av = new ArrayRealVector(A.operate(v));
+        double s = Av.getNorm();
+
+        for (int i = 0, n = u.length; i < n; i++) {
+            u[i] = Av.getEntry(i) / s;
+        }
+
+        return s;
+    }
+
+    /**
+     * Lanczos tridiagonalization for a symmetric matrix C to make s * s tridiagonal matrix T.
+     *
+     * @see http://www.cas.mcmaster.ca/~qiao/publications/spie05.pdf
+     * @param C target symmetric matrix
+     * @param a initial vector
+     * @param T result is stored here
+     */
+    public static void lanczosTridiagonalization(@Nonnull final RealMatrix C,
+            @Nonnull final double[] a, @Nonnull final RealMatrix T) {
+        Preconditions.checkArgument(Arrays.deepEquals(C.getData(), C.transpose().getData()),
+            "Target matrix C must be a symmetric matrix");
+        Preconditions.checkArgument(C.getColumnDimension() == a.length,
+            "Column size of A and length of a should be same");
+        Preconditions.checkArgument(T.getRowDimension() == T.getColumnDimension(),
+            "T must be a square matrix");
+
+        int s = T.getRowDimension();
+
+        // initialize T with zeros
+        T.setSubMatrix(new double[s][s], 0, 0);
+
+        RealVector a0 = new ArrayRealVector(a.length);
+        RealVector r = new ArrayRealVector(a);
+
+        double beta0 = 1.d;
+
+        for (int i = 0; i < s; i++) {
+            RealVector a1 = r.mapDivide(beta0);
+            RealVector Ca1 = C.operate(a1);
+
+            double alpha1 = a1.dotProduct(Ca1);
+
+            r = Ca1.add(a1.mapMultiply(-1.d * alpha1)).add(a0.mapMultiply(-1.d * beta0));
+
+            double beta1 = r.getNorm();
+
+            T.setEntry(i, i, alpha1);
+            if (i - 1 >= 0) {
+                T.setEntry(i, i - 1, beta0);
+            }
+            if (i + 1 < s) {
+                T.setEntry(i, i + 1, beta1);
+            }
+
+            a0 = a1.copy();
+            beta0 = beta1;
+        }
+    }
+
+    /**
+     * QR decomposition for a tridiagonal matrix T.
+     *
+     * @see https://gist.github.com/lightcatcher/8118181
+     * @see http://www.ericmart.in/blog/optimizing_julia_tridiag_qr
+     * @param T target tridiagonal matrix
+     * @param R output matrix for R which is the same shape as T
+     * @param Qt output matrix for Q.T which is the same shape an T
+     */
+    public static void tridiagonalQR(@Nonnull final RealMatrix T, @Nonnull final RealMatrix R,
+            @Nonnull final RealMatrix Qt) {
+        int n = T.getRowDimension();
+        Preconditions.checkArgument(n == R.getRowDimension() && n == R.getColumnDimension(),
+            "T and R must be the same shape");
+        Preconditions.checkArgument(n == Qt.getRowDimension() && n == Qt.getColumnDimension(),
+            "T and Qt must be the same shape");
+
+        // initial R = T
+        R.setSubMatrix(T.getData(), 0, 0);
+
+        // initial Qt = identity
+        Qt.setSubMatrix(eye(n), 0, 0);
+
+        for (int i = 0; i < n - 1; i++) {
+            // Householder projection for a vector x
+            // https://en.wikipedia.org/wiki/Householder_transformation
+            RealVector x = T.getSubMatrix(i, i + 1, i, i).getColumnVector(0);
+            x = unitL2norm(x);
+
+            RealMatrix subR = R.getSubMatrix(i, i + 1, 0, n - 1);
+            R.setSubMatrix(subR.subtract(x.outerProduct(subR.preMultiply(x)).scalarMultiply(2))
+                               .getData(), i, 0);
+
+            RealMatrix subQt = Qt.getSubMatrix(i, i + 1, 0, n - 1);
+            Qt.setSubMatrix(subQt.subtract(x.outerProduct(subQt.preMultiply(x)).scalarMultiply(2))
+                                 .getData(), i, 0);
+        }
+    }
+
+    @Nonnull
+    static RealVector unitL2norm(@Nonnull final RealVector x) {
+        double x0 = x.getEntry(0);
+        double sign = MathUtils.sign(x0);
+        x.setEntry(0, x0 + sign * x.getNorm());
+        return x.unitVector();
+    }
+
+    /**
+     * Find eigenvalues and eigenvectors of given tridiagonal matrix T.
+     *
+     * @see http://web.csulb.edu/~tgao/math423/s94.pdf
+     * @see http://stats.stackexchange.com/questions/20643/finding-matrix-eigenvectors-using-qr-decomposition
+     * @param T target tridiagonal matrix
+     * @param nIter number of iterations for the QR method
+     * @param eigvals eigenvalues are stored here
+     * @param eigvecs eigenvectors are stored here
+     */
+    public static void tridiagonalEigen(@Nonnull final RealMatrix T, @Nonnull final int nIter,
+            @Nonnull final double[] eigvals, @Nonnull final RealMatrix eigvecs) {
+        Preconditions.checkArgument(Arrays.deepEquals(T.getData(), T.transpose().getData()),
+            "Target matrix T must be a symmetric (tridiagonal) matrix");
+        Preconditions.checkArgument(eigvecs.getRowDimension() == eigvecs.getColumnDimension(),
+            "eigvecs must be a square matrix");
+        Preconditions.checkArgument(T.getRowDimension() == eigvecs.getRowDimension(),
+            "T and eigvecs must be the same shape");
+        Preconditions.checkArgument(eigvals.length == eigvecs.getRowDimension(),
+            "Number of eigenvalues and eigenvectors must be same");
+
+        int nEig = eigvals.length;
+
+        // initialize eigvecs as an identity matrix
+        eigvecs.setSubMatrix(eye(nEig), 0, 0);
+
+        RealMatrix T_ = T.copy();
+
+        for (int i = 0; i < nIter; i++) {
+            // QR decomposition for the tridiagonal matrix T
+            RealMatrix R = new Array2DRowRealMatrix(nEig, nEig);
+            RealMatrix Qt = new Array2DRowRealMatrix(nEig, nEig);
+            tridiagonalQR(T_, R, Qt);
+
+            RealMatrix Q = Qt.transpose();
+            T_ = R.multiply(Q);
+            eigvecs.setSubMatrix(eigvecs.multiply(Q).getData(), 0, 0);
+        }
+
+        // diagonal elements correspond to the eigenvalues
+        for (int i = 0; i < nEig; i++) {
+            eigvals[i] = T_.getEntry(i, i);
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/9c0aae3d/core/src/test/java/hivemall/anomaly/SingularSpectrumTransformTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/anomaly/SingularSpectrumTransformTest.java b/core/src/test/java/hivemall/anomaly/SingularSpectrumTransformTest.java
new file mode 100644
index 0000000..5f3f7c0
--- /dev/null
+++ b/core/src/test/java/hivemall/anomaly/SingularSpectrumTransformTest.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package hivemall.anomaly;
+
+import hivemall.anomaly.SingularSpectrumTransformUDF.Parameters;
+import hivemall.anomaly.SingularSpectrumTransformUDF.ScoreFunction;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.zip.GZIPInputStream;
+
+import javax.annotation.Nonnull;
+
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class SingularSpectrumTransformTest {
+    private static final boolean DEBUG = false;
+
+    @Test
+    public void testSVDSST() throws IOException, HiveException {
+        int numChangepoints = detectSST(ScoreFunction.svd, 0.95d);
+        Assert.assertTrue("#changepoints SHOULD be greater than 0: " + numChangepoints,
+            numChangepoints > 0);
+        Assert.assertTrue("#changepoints SHOULD be less than 5: " + numChangepoints,
+            numChangepoints < 5);
+    }
+
+    @Test
+    public void testIKASST() throws IOException, HiveException {
+        int numChangepoints = detectSST(ScoreFunction.ika, 0.65d);
+        Assert.assertTrue("#changepoints SHOULD be greater than 0: " + numChangepoints,
+            numChangepoints > 0);
+        Assert.assertTrue("#changepoints SHOULD be less than 5: " + numChangepoints,
+            numChangepoints < 5);
+    }
+
+    @Test
+    public void testSVDTwitterData() throws IOException, HiveException {
+        int numChangepoints = detectTwitterData(ScoreFunction.svd, 0.005d);
+        Assert.assertTrue("#changepoints SHOULD be greater than 0: " + numChangepoints,
+            numChangepoints > 0);
+        Assert.assertTrue("#changepoints SHOULD be less than 5: " + numChangepoints,
+            numChangepoints < 5);
+    }
+
+    @Test
+    public void testIKATwitterData() throws IOException, HiveException {
+        int numChangepoints = detectTwitterData(ScoreFunction.ika, 0.0175d);
+        Assert.assertTrue("#changepoints SHOULD be greater than 0: " + numChangepoints,
+            numChangepoints > 0);
+        Assert.assertTrue("#changepoints SHOULD be less than 5: " + numChangepoints,
+            numChangepoints < 5);
+    }
+
+    private static int detectSST(@Nonnull final ScoreFunction scoreFunc,
+            @Nonnull final double threshold) throws IOException, HiveException {
+        Parameters params = new Parameters();
+        params.set(scoreFunc);
+        PrimitiveObjectInspector oi = PrimitiveObjectInspectorFactory.javaDoubleObjectInspector;
+        SingularSpectrumTransform sst = new SingularSpectrumTransform(params, oi);
+        double[] outScores = new double[1];
+
+        BufferedReader reader = readFile("cf1d.csv");
+        println("x change");
+        String line;
+        int numChangepoints = 0;
+        while ((line = reader.readLine()) != null) {
+            double x = Double.parseDouble(line);
+            sst.update(x, outScores);
+            printf("%f %f%n", x, outScores[0]);
+            if (outScores[0] > threshold) {
+                numChangepoints++;
+            }
+        }
+
+        return numChangepoints;
+    }
+
+    private static int detectTwitterData(@Nonnull final ScoreFunction scoreFunc,
+            @Nonnull final double threshold) throws IOException, HiveException {
+        Parameters params = new Parameters();
+        params.set(scoreFunc);
+        PrimitiveObjectInspector oi = PrimitiveObjectInspectorFactory.javaDoubleObjectInspector;
+        SingularSpectrumTransform sst = new SingularSpectrumTransform(params, oi);
+        double[] outScores = new double[1];
+
+        BufferedReader reader = readFile("twitter.csv.gz");
+        println("# time x change");
+        String line;
+        int i = 1, numChangepoints = 0;
+        while ((line = reader.readLine()) != null) {
+            double x = Double.parseDouble(line);
+            sst.update(x, outScores);
+            printf("%d %f %f%n", i, x, outScores[0]);
+            if (outScores[0] > threshold) {
+                numChangepoints++;
+            }
+            i++;
+        }
+
+        return numChangepoints;
+    }
+
+    private static void println(String msg) {
+        if (DEBUG) {
+            System.out.println(msg);
+        }
+    }
+
+    private static void printf(String format, Object... args) {
+        if (DEBUG) {
+            System.out.printf(format, args);
+        }
+    }
+
+    @Nonnull
+    private static BufferedReader readFile(@Nonnull String fileName) throws IOException {
+        InputStream is = SingularSpectrumTransformTest.class.getResourceAsStream(fileName);
+        if (fileName.endsWith(".gz")) {
+            is = new GZIPInputStream(is);
+        }
+        return new BufferedReader(new InputStreamReader(is));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/9c0aae3d/core/src/test/java/hivemall/utils/lang/PreconditionsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/lang/PreconditionsTest.java b/core/src/test/java/hivemall/utils/lang/PreconditionsTest.java
index b0cfbd0..4cbbd3f 100644
--- a/core/src/test/java/hivemall/utils/lang/PreconditionsTest.java
+++ b/core/src/test/java/hivemall/utils/lang/PreconditionsTest.java
@@ -20,6 +20,7 @@ package hivemall.utils.lang;
 
 import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.junit.Assert;
 import org.junit.Test;
 
 public class PreconditionsTest {
@@ -29,9 +30,35 @@ public class PreconditionsTest {
         Preconditions.checkNotNull(null, UDFArgumentException.class);
     }
 
+    @Test
+    public void testCheckNotNullTClassOfE2() {
+        final String msg = "safdfvzfd";
+        try {
+            Preconditions.checkNotNull(null, msg, UDFArgumentException.class);
+        } catch (UDFArgumentException e) {
+            if (e.getMessage().equals(msg)) {
+                return;
+            }
+        }
+        Assert.fail("should not reach");
+    }
+
     @Test(expected = HiveException.class)
-    public void testCheckArgumentBooleanClassOfE() throws UDFArgumentException, HiveException {
+    public void testCheckArgumentBooleanClassOfE() throws HiveException {
         Preconditions.checkArgument(false, HiveException.class);
     }
 
+    @Test
+    public void testCheckArgumentBooleanClassOfE2() {
+        final String msg = "safdfvzfd";
+        try {
+            Preconditions.checkArgument(false, msg, HiveException.class);
+        } catch (HiveException e) {
+            if (e.getMessage().equals(msg)) {
+                return;
+            }
+        }
+        Assert.fail("should not reach");
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/9c0aae3d/core/src/test/java/hivemall/utils/math/MatrixUtilsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/math/MatrixUtilsTest.java b/core/src/test/java/hivemall/utils/math/MatrixUtilsTest.java
index e122fbf..8b3053a 100644
--- a/core/src/test/java/hivemall/utils/math/MatrixUtilsTest.java
+++ b/core/src/test/java/hivemall/utils/math/MatrixUtilsTest.java
@@ -20,6 +20,7 @@ package hivemall.utils.math;
 
 import org.apache.commons.math3.linear.Array2DRowRealMatrix;
 import org.apache.commons.math3.linear.RealMatrix;
+import org.apache.commons.math3.linear.SingularValueDecomposition;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -206,4 +207,70 @@ public class MatrixUtilsTest {
         Assert.assertArrayEquals(expected, actual);
     }
 
+    @Test
+    public void testPower1() {
+        RealMatrix A = new Array2DRowRealMatrix(new double[][] {new double[] {1, 2, 3},
+                new double[] {4, 5, 6}});
+
+        double[] x = new double[3];
+        x[0] = Math.random();
+        x[1] = Math.random();
+        x[2] = Math.random();
+
+        double[] u = new double[2];
+        double[] v = new double[3];
+
+        double s = MatrixUtils.power1(A, x, 2, u, v);
+
+        SingularValueDecomposition svdA = new SingularValueDecomposition(A);
+
+        Assert.assertArrayEquals(svdA.getU().getColumn(0), u, 0.001d);
+        Assert.assertArrayEquals(svdA.getV().getColumn(0), v, 0.001d);
+        Assert.assertEquals(svdA.getSingularValues()[0], s, 0.001d);
+    }
+
+    @Test
+    public void testLanczosTridiagonalization() {
+        // Symmetric matrix
+        RealMatrix C = new Array2DRowRealMatrix(new double[][] {new double[] {1, 2, 3, 4},
+                new double[] {2, 1, 4, 3}, new double[] {3, 4, 1, 2}, new double[] {4, 3, 2, 1}});
+
+        // naive initial vector
+        double[] a = new double[] {1, 1, 1, 1};
+
+        RealMatrix actual = new Array2DRowRealMatrix(new double[4][4]);
+        MatrixUtils.lanczosTridiagonalization(C, a, actual);
+
+        RealMatrix expected = new Array2DRowRealMatrix(new double[][] {new double[] {40, 60, 0, 0},
+                new double[] {60, 10, 120, 0}, new double[] {0, 120, 10, 120},
+                new double[] {0, 0, 120, 10}});
+
+        Assert.assertEquals(expected, actual);
+    }
+
+    @Test
+    public void testTridiagonalEigen() {
+        // Tridiagonal Matrix
+        RealMatrix T = new Array2DRowRealMatrix(new double[][] {new double[] {40, 60, 0, 0},
+                new double[] {60, 10, 120, 0}, new double[] {0, 120, 10, 120},
+                new double[] {0, 0, 120, 10}});
+
+        double[] eigvals = new double[4];
+        RealMatrix eigvecs = new Array2DRowRealMatrix(new double[4][4]);
+
+        MatrixUtils.tridiagonalEigen(T, 2, eigvals, eigvecs);
+
+        RealMatrix actual = eigvecs.multiply(eigvecs.transpose());
+
+        RealMatrix expected = new Array2DRowRealMatrix(new double[4][4]);
+        for (int i = 0; i < 4; i++) {
+            expected.setEntry(i, i, 1);
+        }
+
+        for (int i = 0; i < 4; i++) {
+            for (int j = 0; j < 4; j++) {
+                Assert.assertEquals(expected.getEntry(i, j), actual.getEntry(i, j), 0.001d);
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/9c0aae3d/resources/eclipse-style.xml
----------------------------------------------------------------------
diff --git a/resources/eclipse-style.xml b/resources/eclipse-style.xml
index a26e895..8cb5eb5 100644
--- a/resources/eclipse-style.xml
+++ b/resources/eclipse-style.xml
@@ -64,7 +64,7 @@
 <setting id="org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation" value="do not insert"/>
 <setting id="org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments" value="do not insert"/>
 <setting id="org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch" value="insert"/>
-<setting id="org.eclipse.jdt.core.formatter.comment.line_length" value="100"/>
+<setting id="org.eclipse.jdt.core.formatter.comment.line_length" value="150"/>
 <setting id="org.eclipse.jdt.core.formatter.use_on_off_tags" value="false"/>
 <setting id="org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression" value="do not insert"/>
 <setting id="org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant" value="insert"/>