You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemds.apache.org by mb...@apache.org on 2020/09/12 20:51:44 UTC

[systemds] branch master updated: [SYSTEMDS-2461] Updated slicefinder builtin (implementation, tests)

This is an automated email from the ASF dual-hosted git repository.

mboehm7 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/systemds.git


The following commit(s) were added to refs/heads/master by this push:
     new bc858f6  [SYSTEMDS-2461] Updated slicefinder builtin (implementation, tests)
bc858f6 is described below

commit bc858f65e910375922c5981e160b5cb1ada0e437
Author: Matthias Boehm <mb...@gmail.com>
AuthorDate: Sat Sep 12 22:51:15 2020 +0200

    [SYSTEMDS-2461] Updated slicefinder builtin (implementation, tests)
    
    This patch graduates the new slice finding implementation in dml from
    staging to builtin and thus, replaces the previous experimental
    slicefinder implementation. Accordingly, we also modified the tests to
    fit the new scoring function, and work with a tiny, real dataset.
---
 scripts/builtin/slicefinder.dml                    | 465 +++++++++++----------
 .../functions/builtin/BuiltinSliceFinderTest.java  | 200 +++------
 .../scripts/functions/builtin/data/Salaries.csv    | 398 ++++++++++++++++++
 src/test/scripts/functions/builtin/slicefinder.dml |  26 +-
 4 files changed, 725 insertions(+), 364 deletions(-)

diff --git a/scripts/builtin/slicefinder.dml b/scripts/builtin/slicefinder.dml
index 4996cca..6ec529a 100644
--- a/scripts/builtin/slicefinder.dml
+++ b/scripts/builtin/slicefinder.dml
@@ -20,253 +20,276 @@
 #-------------------------------------------------------------
 
 #-------------------------------------------------------------
-# X    Input matrix
-# W    beta in lm
-# Y    matrix column for training
-# k    top-K subsets / slices
-# paq  amount of values wanted for each col, if = 1 then its off
-# S    amount of subsets to combine (for now supported only 2/1)
+# X         Input matrix (integer encoded [1..v])
+# e         error vector (classification accuracy, l2 norm, etc)
+# k         top-K subsets / slices
+# minSup    minimum support (min number of rows per slice)
+# alpha     weight [0,1]: 0 only size, 1 only error
+# dpEval    flag for data-parallel slice evaluation, 
+#           otherwise task-parallel
+# verbose   flag for verbose debug output 
 # ------------------------------------------------------------
+# TK        top-k slices (k x ncol(X) if successful) 
+# TKC       score, size, error of slices (k x 3)
+# ------------------------------------------------------------
+
+m_slicefinder = function(Matrix[Double] X, Matrix[Double] e,
+    Integer k = 4, Integer minSup = 32, Double alpha = 0.5,
+    Boolean dpEval = FALSE, Boolean verbose = FALSE)
+  return(Matrix[Double] TK, Matrix[Double] TKC)
+{
+  m = nrow(X);
+  n = ncol(X);
+
+  # prepare offset vectors and one-hot encoded X
+  fdom = colMaxs(X);
+  foffb = t(cumsum(t(fdom))) - fdom;
+  foffe = t(cumsum(t(fdom)))
+  rix = matrix(seq(1,m)%*%matrix(1,1,n), m*n, 1)
+  cix = matrix(X + foffb, m*n, 1);
+  X2 = table(rix, cix); #one-hot encoded
+
+  # initialize statistics and basic slices
+  n2 = ncol(X2);     # one-hot encoded features
+  eAvg = sum(e) / m; # average error
+  [S, R] = createAndScoreBasicSlices(X2, e, eAvg, minSup, alpha, verbose); 
+
+  # initialize top-k
+  [TK, TKC] = maintainTopK(S, R, matrix(0, 0, n2), matrix(0, 0, 3), k, minSup);
+
+  if( verbose ) {
+    [maxsc, minsc] = analyzeTopK(TKC);
+    print("SliceFinder: initial top-K: count="+nrow(TK)+", max="+maxsc+", min="+minsc)
+  }
+
+  # lattice enumeration w/ size/error pruning, one iteration per level
+  # termination condition (max #feature levels)
+  level = 1;
+  while( nrow(S) > 0 & sum(S) > 0 & level < n ) {
+    level = level + 1;
 
-m_slicefinder = function(Matrix[Double] X, Matrix[Double] W, Matrix[Double] Y, Integer k = 1, Integer paq = 1, Integer S = 2) return(Matrix[Double] result) {
-
-  X0 = cbind(X, Y);
-  beta = W;
-  col = ncol(X0);
-  row = nrow(X0);
-
-  val_matrix = matrix(0, rows = 2, cols = col - 1);
-  vcol = ncol(val_matrix);
-  empty_row = matrix(0, rows = 1, cols = col - 1);
-
-  #first scan, making val_matrix with different values from the each col.
-  #first row or this matrix indicates how many different values are in each col.
-
-  for (j in 1:col - 1) {
-    vector = order(target = X0[, j], by = 1, decreasing = FALSE);
-    val_matrix[2, j] = vector[1, 1];
-    val_counter = 1;
-    for (i in 1:row) {
-      if (as.scalar(val_matrix[val_counter + 1, j]) != as.scalar(vector[i, 1])) {
-        if (nrow(val_matrix) == val_counter + 1)
-          val_matrix = rbind(val_matrix, empty_row);
-        val_counter = val_counter + 1;
-        val_matrix[val_counter + 1, j] = vector[i, 1];
-      }
+    # enumerate candidate join pairs, incl size/error pruning 
+    nrS = nrow(S);
+    S = getPairedCandidates(S, R, TK, TKC, k, level, eAvg, minSup, alpha, n2, foffb, foffe); 
+
+    if(verbose) {
+      print("\nSliceFinder: level "+level+":")
+      print(" -- generated paired slice candidates: "+nrS+" -> "+nrow(S));
     }
 
-    val_matrix[1, j] = val_counter;
-
-    #here I add some condition to split the values from each column if val_counter is too big;
-    ################################################
-    #this code relates to large datasets
-    #packing values according to paq value
-    ## TODO -- this if needs to be checked, is not working properly with all the paq values
-    if (paq != 1) {
-
-      position = floor(val_counter / paq);
-      for (a in 1:paq) {
-        if (a == paq) {
-          pos = as.scalar(val_matrix[1, j]) + 1;
-          tresh = val_matrix[pos, j];
-          val_matrix[a + 1, j] = tresh;
-        } else {
-          pos = position * a;
-          tresh = val_matrix[pos, j];
-          val_matrix[a + 1, j] = tresh;
-        }
-      }
-
-      val_matrix = val_matrix[1:paq + 1,];
+    # extract and evaluate candidate slices
+    if( dpEval ) { #data-parallel
+      R = evalSlice(X2, e, eAvg, t(S), level, alpha);
+    }
+    else { # task-parallel
+      R = matrix(0, nrow(S), 3)
+      parfor( i in 1:nrow(S) )
+        R[i,] = evalSlice(X2, e, eAvg, t(S[i,]), level, alpha);
+    }
 
+    # maintain top-k after evaluation
+    [TK, TKC] = maintainTopK(S, R, TK, TKC, k, minSup);
+
+    if(verbose) {
+      [maxsc, minsc] = analyzeTopK(TKC);
+      valid = as.integer(sum(R[,3]>=minSup));
+      print(" -- valid slices after eval: "+valid+"/"+nrow(S));
+      print(" -- top-K: count="+nrow(TK)+", max="+maxsc+", min="+minsc);
     }
-    ##################################################
   }
-  vrow = nrow(val_matrix);
-  vcol = ncol(val_matrix);
-  totalrows = (vrow - 1) * vcol;
 
-  #######################################
-  Y0 = X0[1:nrow(X0), ncol(X0)];
-  Y = lmpredict(X = X0[1:nrow(X0), 1:col - 1], w = beta, icpt = 0);
-  [error0, diff0] = standart_error(Y, Y0);
-  #####################################################
-  # set_matrix will be the matrix with all slices and combination of them
-  #acctually supporting only combination of 2 slices
-  set_matrix = matrix(0, rows = 1, cols = 2 + (9 * S));
-  set_row = matrix(0, rows = 1, cols = 2 + (9 * S));
+  TK = decodeTopK(TK, foffb, foffe);
 
+  if( verbose ) {
+    print("SliceFinder: terminated at level "+level+":\n"
+      + toString(TK) + "\n" + toString(TKC));
+  }
+}
 
-  # first_slices is returning in slice_matrix single subsets
+createAndScoreBasicSlices = function(Matrix[Double] X2, Matrix[Double] e, 
+    Double eAvg, Double minSup, Double alpha, Boolean verbose)
+  return(Matrix[Double] S, Matrix[Double] R)
+{
+  n2 = ncol(X2);
+  cCnts = t(colSums(X2)); # column counts
+  err = t(t(e) %*% X2)    # total error vector
+
+  if( verbose ) {
+    drop = as.integer(sum(cCnts < minSup));
+    print("SliceFinder: dropping "+drop+"/"+n2+" features below minSup = "+minSup+".");
+  }
 
-  set_matrix = first_slices(val_matrix, set_matrix, X0, set_row, beta, paq, S);
+  # working set of active slices (#attr x #slices) and top k
+  selCols = (cCnts >= minSup);
+  attr = removeEmpty(target=seq(1,n2), margin="rows", select=selCols);
+  ss = removeEmpty(target=cCnts, margin="rows", select=selCols);
+  se = removeEmpty(target=err, margin="rows", select=selCols);
+  S = table(seq(1,nrow(attr)), attr, nrow(attr), n2);
 
-  #double_features returns subsets that cover 2 values from the same or different feature
-  if (S == 2)
-  set_matrix = double_features(val_matrix, set_matrix, X0, Y, set_row, beta, paq);
+  # score 1-slices and create initial top-k 
+  sc = score(ss, se, eAvg, alpha, nrow(X2));
+  R = cbind(sc, se, ss);
+}
 
-  ress = order(target = set_matrix, by = 1, decreasing = TRUE);
-  set_rows = nrow(set_matrix);
-  set_cols = ncol(set_matrix);
+score = function(Matrix[Double] ss, Matrix[Double] se, Double eAvg, Double alpha, Integer n)
+  return(Matrix[Double] sc)
+{
+  sc = alpha * ((se/ss) / eAvg - 1) - (1-alpha) * (n/ss - 1);
+}
 
-  #checking values by ordering set_matrix col 1 or 2
-  result = ress[1:k,];
+scoreUB = function(Matrix[Double] ss, Matrix[Double] se, 
+    Double eAvg, Integer minSup, Double alpha, Integer n)
+  return(Matrix[Double] sc)
+{
+  sc = alpha * ((se/minSup) / eAvg - 1) - (1-alpha) * (n/ss - 1);
 }
 
-standart_error = function(matrix[double] Y, matrix[double] Y0) return(double error, double diff) {
-  diff = var(Y0 - Y);
-  error = sqrt(sum((Y0 - Y) ^ 2) / (nrow(Y) - 2));
+
+maintainTopK = function(Matrix[Double] S, Matrix[Double] R, 
+    Matrix[Double] TK, Matrix[Double] TKC, Integer k, Integer minSup) 
+  return(Matrix[Double] TK, Matrix[Double] TKC)
+{
+  # prune invalid minSup and scores
+  I = (R[,1] > 0) & (R[,3] >= minSup);
+
+  if( sum(I)!=0 ) {
+    S = removeEmpty(target=S, margin="rows", select=I);
+    R = removeEmpty(target=R, margin="rows", select=I);
+
+    # evaluated candidated and previous top-k
+    slices = rbind(TK, S);
+    scores = rbind(TKC, R);
+
+    # extract top-k
+    IX = order(target=scores, by=1, decreasing=TRUE, index.return=TRUE);
+    IX = IX[1:min(k,nrow(IX)),];
+    P = table(seq(1,nrow(IX)), IX, nrow(IX), nrow(slices));
+    TK = P %*% slices;
+    TKC = P %*% scores;
+  }
 }
 
-#index = binary search
-index = function(matrix[double] X, Integer column, double value, Integer mode) return(Integer pos) {
-  begin = 1;
-  e = nrow(X) + 1;
-  while (begin < e - 1) {
-    pos = as.integer(floor((begin + e) / 2));
-    if (mode == 0) {
-      if (as.scalar(X[pos, column]) < value)
-        begin = pos;
-      else
-        e = pos;
-    }
-    else if (mode == 1) {
-      if (as.scalar(X[pos, column]) <= value)
-        begin = pos;
-      else
-        e = pos;
-    }
+analyzeTopK = function(Matrix[Double] TKC) return(Double maxsc, Double minsc) {
+  maxsc = -Inf;
+  minsc = -Inf;
+  if( nrow(TKC)>0 ) {
+    maxsc = as.scalar(TKC[1,1]);
+    minsc = as.scalar(TKC[nrow(TKC),1]);
   }
 }
 
-first_slices = function(Matrix[Double] val_matrix, Matrix[Double] set_matrix, Matrix[Double] X0, Matrix[Double] set_row, Matrix[Double] beta, Integer paq, Integer S) return(Matrix[Double] set_matrix) {
-  col = ncol(X0);
-  row = nrow(X0);
-  vrow = nrow(val_matrix);
-  vcol = ncol(val_matrix);
-  cont = nrow(set_matrix);
-  b0 = 1;
-  b1 = col - 1;
-
-  for (j in 1:vcol) {
-    num_value = as.scalar(val_matrix[1, j]);
-
-    if (paq != 1)
-      num_value = paq;
-    x = order(target = X0, by = j, decreasing = FALSE);
-    
-    for (i in 2:num_value + 1) {
-      value1 = as.scalar(val_matrix[i, j]);
-
-      if (paq != 1) {
-        if (i == 2) {
-          a0 = 1;
-          swich = 1;
-          value0 = value1;
-        }
-        else if (as.scalar(val_matrix[i - 1, j]) <= as.scalar(val_matrix[i, j])) {
-          value0 = as.scalar(val_matrix[i - 1, j]);
-          a0 = index(x, j, value0, 1);
-          swich = 1;
-        }
-      }
-      else {
-        swich = 1;
-        value0 = value1;
-        a0 = index(x, j, value0, 0);
-      }
-
-      if (nrow(set_matrix) < cont)
-        set_matrix = rbind(set_matrix, set_row);
-
-      if (swich == 1) {
-        a1 = index(x, j, value1, 1);
-        slice_matrix = x[a0:a1, b0:b1];
-        Y0 = x[a0:a1, col];
-        Y = lmpredict(X = slice_matrix, w = beta, icpt = 0);
-        [error, diff] = standart_error(Y, Y0);
-        ## TODO -  mylist needs to be modified in order to show the total rows of the slice
-        if (S == 1)
-          mylist = as.matrix(list(diff, error, value0, value1, j, nrow(slice_matrix), ncol(slice_matrix), a0, a1, b0, b1))
-        else
-          mylist = as.matrix(list(diff, error, value0, value1, j, nrow(slice_matrix), ncol(slice_matrix), a0, a1, b0, b1, 0, 0, 0, 0, 0, 0, 0, 0, 0))
-
-        set_matrix[cont, 1:ncol(set_matrix)] = t(mylist)
-        cont = cont + 1;
-        swich = 0;
-      }
+getPairedCandidates = function(Matrix[Double] S, Matrix[Double] R, 
+    Matrix[Double] TK, Matrix[Double] TKC, Integer k, Integer level, 
+    Double eAvg, Integer minSup, Double alpha, Integer n2, 
+    Matrix[Double] foffb, Matrix[Double] foffe)
+  return(Matrix[Double] P) 
+{
+  # prune invalid slices (possible without affecting overall
+  # pruning effectiveness due to handling of missing parents)
+  pI = (R[,3] >= minSup);
+  S = removeEmpty(target=S, margin="rows", select=pI)
+  R = removeEmpty(target=R, margin="rows", select=pI)
+
+  # join compatible slices (without self)
+  join = S %*% t(S) == (level-2)
+  I = upper.tri(target=join, diag=FALSE, values=TRUE);
+  
+  # pair construction
+  nr = nrow(I); nc = ncol(I);
+  rix = matrix(I * seq(1,nr), nr*nc, 1);
+  cix = matrix(I * t(seq(1,nc)), nr*nc, 1);
+  rix = removeEmpty(target=rix, margin="rows");
+  cix = removeEmpty(target=cix, margin="rows");
+  
+  P = matrix(0,0,ncol(S))
+  if( sum(rix)!=0 ) {
+    P1 = table(seq(1,nrow(rix)), rix, nrow(rix), nrow(S));
+    P2 = table(seq(1,nrow(cix)), cix, nrow(rix), nrow(S));
+    P12 = P1 + P2; # combined slice
+    P = (P12 %*% S) != 0;
+    ss = min(P1 %*% R[,3], P2 %*% R[,3])
+    se = min(P1 %*% R[,2], P2 %*% R[,2])
+
+    # prune invalid self joins (>1 bit per feature)
+    I = matrix(1, nrow(P), 1);
+    for( j in 1:ncol(foffb) ) {
+      beg = as.scalar(foffb[1,j])+1;
+      end = as.scalar(foffe[1,j]);
+      I = I & (rowSums(P[,beg:end]) <= 1);
+    }
+    P12 = removeEmpty(target=P12, margin="rows", select=I)
+    P = removeEmpty(target=P, margin="rows", select=I);
+    ss = removeEmpty(target=ss, margin="rows", select=I);
+    se = removeEmpty(target=se, margin="rows", select=I);
+
+    # prepare IDs for deduplication and pruning
+    ID = matrix(0, nrow(P), 1);
+    dom = foffe-foffb+1;
+    for( j in 1:ncol(dom) ) {
+      beg = as.scalar(foffb[1,j])+1;
+      end = as.scalar(foffe[1,j]);
+      I = rowIndexMax(P[,beg:end]) * rowMaxs(P[,beg:end]);
+      prod = 1;
+      if(j<ncol(dom))
+        prod = prod(dom[1,(j+1):ncol(dom)])
+      ID = ID + I * prod;
     }
+
+    # ID transformation to avoid exceeding INT_MAX and
+    # and to void creating huge sparse intermediates
+    [ID, M] = transformencode(target=as.frame(ID), spec="{ids:true,recode:[1]}")
+
+    # size pruning, with rowMin-rowMax transform 
+    # to avoid densification (ignored zeros)
+    map = table(ID, seq(1,nrow(P)), max(ID), nrow(P))
+    ubSizes = 1/rowMaxs(map * (1/t(ss)));
+    ubSizes = replace(target=ubSizes, pattern=Inf, replacement=0);
+    fSizes = (ubSizes >= minSup)
+
+    # error pruning
+    ubError = 1/rowMaxs(map * (1/t(se)));
+    ubError = replace(target=ubError, pattern=Inf, replacement=0);
+    ubScores = scoreUB(ubSizes, ubError, eAvg, minSup, alpha, n2);
+    [maxsc, minsc] = analyzeTopK(TKC);
+    fScores = (ubScores > minsc & ubScores > 0) 
+
+    # missing parents pruning
+    numParents = rowSums((map %*% P12) != 0) 
+    fParents = (numParents == level);
+
+    # apply all pruning 
+    map = map * (fSizes & fScores & fParents);
+    
+    # deduplication of join outputs
+    Dedup = removeEmpty(target=map, margin="rows") != 0
+    P = (Dedup %*% P) != 0
   }
 }
 
+evalSlice = function(Matrix[Double] X, Matrix[Double] e, Double eAvg, 
+    Matrix[Double] tS, Integer l, Double alpha) 
+  return(Matrix[Double] R)
+{
+  I = (X %*% tS) == l; # slice indicator
+  ss = t(colSums(I));  # absolute slice size (nnz)
+  se = t(t(e) %*% I);  # absolute slice error 
+
+  # score of relative error and relative size
+  sc = score(ss, se, eAvg, alpha, nrow(X));
+  R = cbind(sc, se, ss);
+}
 
-double_features = function(Matrix[Double] val_matrix, Matrix[Double] set_matrix, Matrix[Double] X0, Matrix[Double] Y, Matrix[Double] set_row, Matrix[Double] beta, Integer paq) return(Matrix[Double] set_matrix) {
-
-  vrow = nrow(val_matrix);
-  vcol = ncol(val_matrix);
-  cont = nrow(set_matrix);
-  col = ncol(X0);
-  row = nrow(X0);
-  totalrows = (vrow - 1) * vcol;
-  b0 = 1;
-  b1 = col - 1;
-  slice_number = 2;
-
-  #combining subsets from set_matrix with the ones from val_matrix
-  #avoiding repeating subsets, taking in account the amount of values in val_matrix or the paq value if activated.
-  #new subsets checked are stored in set_matrix
-
-  for (j in 1:vcol) {
-    num_value = as.scalar(val_matrix[1, j]);
-    x = order(target = X0, by = j, decreasing = FALSE);
-
-    if (paq != 1)
-      num_value = paq;
-    if (j == num_value + 1)
-      vrow = vrow - 1;
-
-    for (i in 2:num_value + 1) {
-      if (i > 2 | j > 1)
-        slice_number = slice_number + 1;
-
-      for (a in slice_number:totalrows) {
-        num_col = as.scalar(set_matrix[a, 5]);
-        x_x = order(target = X0, by = num_col, decreasing = FALSE);
-
-        value_A0 = as.scalar(set_matrix[a, 3]);
-        value_A1 = as.scalar(set_matrix[a, 4]);
-        a00 = as.scalar(set_matrix[a, 8]);
-        a11 = as.scalar(set_matrix[a, 9]);
-        A = x_x[a00:a11, b0:b1];
-        Ya = x_x[a00:a11, col];
-
-        if (nrow(set_matrix) <= cont)
-          set_matrix = rbind(set_matrix, set_row);
-
-        value_B1 = as.scalar(val_matrix[i, j]);
-
-        if (i == 2) {
-          a0 = 1;
-          value_B0 = value_B1;
-        }
-        else if (as.scalar(val_matrix[i - 1, j]) <= as.scalar(val_matrix[i, j])) {
-          value_B0 = as.scalar(val_matrix[i - 1, j]);
-          a0 = index(x, j, value_B0, 1);
-        }
-
-        a1 = index(x, j, value_B1, 1);
-        B = x[a0:a1, b0:b1];
-        slice_matrix = rbind(A, B);
-        Yb = x[a0:a1, col];
-
-        Y0 = rbind(Ya, Yb);
-        Y = lmpredict(X = slice_matrix, w = beta, icpt = 0);
-        [error, diff] = standart_error(Y, Y0);
-        ## TODO - next code needs to be modified in order to show the total rows of the slice (as in previous function)
-        set_matrix[cont, 1:ncol(set_matrix)] = t(as.matrix(list(diff, error, value_A0, value_A1, num_col, nrow(A),
-          ncol(A), a00, a11, b0, b1, value_B0, value_B1, j, nrow(B), ncol(B), a0, a1, b0, b1)));
-        cont = cont + 1;
-      }
+decodeTopK = function(Matrix[Double] TK, Matrix[Double] foffb, Matrix[Double] foffe)
+  return(Matrix[Double] TK) 
+{
+  R = matrix(1, nrow(TK), ncol(foffb));
+  if( nrow(TK) > 0 ) {
+    parfor( j in 1:ncol(foffb) ) {
+      beg = as.scalar(foffb[1,j])+1;
+      end = as.scalar(foffe[1,j]);
+      I = rowSums(TK[,beg:end]) * rowIndexMax(TK[,beg:end]);
+      R[, j] = I;
     }
   }
+  TK = R;
 }
diff --git a/src/test/java/org/apache/sysds/test/functions/builtin/BuiltinSliceFinderTest.java b/src/test/java/org/apache/sysds/test/functions/builtin/BuiltinSliceFinderTest.java
index 6bcf37b..ff9b639 100644
--- a/src/test/java/org/apache/sysds/test/functions/builtin/BuiltinSliceFinderTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/builtin/BuiltinSliceFinderTest.java
@@ -20,175 +20,99 @@
 package org.apache.sysds.test.functions.builtin;
 
 
-import java.util.HashMap;
-import java.util.Map.Entry;
-import java.util.Random;
-
 import org.junit.Test;
 import org.apache.sysds.common.Types.ExecMode;
-import org.apache.sysds.lops.LopProperties.ExecType;
-import org.apache.sysds.runtime.matrix.data.MatrixValue;
 import org.apache.sysds.test.AutomatedTestBase;
 import org.apache.sysds.test.TestConfiguration;
 import org.apache.sysds.test.TestUtils;
 
 public class BuiltinSliceFinderTest extends AutomatedTestBase {
 
-	private final static String TEST_NAME = "slicefinder";
-	private final static String TEST_DIR = "functions/builtin/";
+	private static final String TEST_NAME = "slicefinder";
+	private static final String TEST_DIR = "functions/builtin/";
 	private static final String TEST_CLASS_DIR = TEST_DIR + BuiltinSliceFinderTest.class.getSimpleName() + "/";
-
-	private final static int rows = 2000;
-	private final static int cols = 10;
-
+	private static final boolean VERBOSE = true;
+	
+	private static final double[][] EXPECTED_TOPK = new double[][]{
+		{1.042, 69210699988.477, 18.000},
+		{0.478, 92957580467.849, 39.000},
+		{0.316, 40425449547.480, 10.000},
+		{0.262, 67630559163.266, 29.000},
+		{0.224, 202448990843.317, 125.000},
+		{0.218, 68860581248.568, 31.000},
+		{0.164, 206527445340.279, 135.000},
+		{0.122, 68961886413.866, 34.000},
+		{0.098, 360278523220.479, 266.000},
+		{0.092, 73954209826.485, 39.000}
+	};
+	
 	@Override
 	public void setUp() {
-		addTestConfiguration(TEST_NAME, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME, new String[]{"B"}));
+		addTestConfiguration(TEST_NAME, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME, new String[]{"R"}));
 	}
 
 	@Test
-	public void SingleFreatureTest() {
-		runslicefindertest(1,true, ExecType.CP, BuiltinLmTest.LinregType.AUTO);
+	public void testTop4HybridDP() {
+		runSliceFinderTest(4, true, ExecMode.HYBRID);
 	}
-
+	
 	@Test
-	public void MultipleValuesOneFeature() {
-		runslicefindertest(2,true, ExecType.CP, BuiltinLmTest.LinregType.AUTO);
+	public void testTop4SinglenodeDP() {
+		runSliceFinderTest(4, true, ExecMode.SINGLE_NODE);
 	}
-
+	
 	@Test
-	public void MultipleFeaturesSingleValues() {
-		runslicefindertest(3,true, ExecType.CP, BuiltinLmTest.LinregType.AUTO);
+	public void testTop4HybridTP() {
+		runSliceFinderTest(4, false, ExecMode.HYBRID);
+	}
+	
+	@Test
+	public void testTop4SinglenodeTP() {
+		runSliceFinderTest(4, false, ExecMode.SINGLE_NODE);
 	}
 
-	private void runslicefindertest(int test,boolean sparse, ExecType instType, BuiltinLmTest.LinregType linregAlgo) {
-		ExecMode platformOld = setExecMode(instType);
+	@Test
+	public void testTop10HybridDP() {
+		runSliceFinderTest(10, true, ExecMode.HYBRID);
+	}
+	
+	@Test
+	public void testTop10SinglenodeDP() {
+		runSliceFinderTest(10, true, ExecMode.SINGLE_NODE);
+	}
+	
+	@Test
+	public void testTop10HybridTP() {
+		runSliceFinderTest(10, false, ExecMode.HYBRID);
+	}
+	
+	@Test
+	public void testTop10SinglenodeTP() {
+		runSliceFinderTest(10, false, ExecMode.SINGLE_NODE);
+	}
+	
+	private void runSliceFinderTest(int K, boolean dp, ExecMode mode) {
+		ExecMode platformOld = setExecMode(ExecMode.HYBRID);
 		String dml_test_name = TEST_NAME;
 		loadTestConfiguration(getTestConfiguration(TEST_NAME));
 		String HOME = SCRIPT_DIR + TEST_DIR;
-
+		String data = HOME + "/data/Salaries.csv";
+		
 		try {
 			loadTestConfiguration(getTestConfiguration(TEST_NAME));
+			//setOutputBuffering(false);
 			fullDMLScriptName = HOME + dml_test_name + ".dml";
-			programArgs = new String[]{"-args", input("A"), input("B"), input("Y0"), output("C")};
-			double[][] A = TestUtils.ceil(getRandomMatrix(rows, cols, 0, 10, 1, 7));
-			double[][] B = TestUtils.ceil(getRandomMatrix(10, 1, 0, 10, 1.0, 3));
-			double[][] As = new double[rows][cols];
-			double [] Ys = new double[rows];
-			double Y[] = new double[rows];
-			
-			//Y = X %*% B
-			for (int i = 0; i < rows; i++)
-				for (int k = 0; k < cols; k++)
-					Y[i] += A[i][k] * B[k][0];
-
-			double AA[][] = new double[rows][cols+1];
-
-			int value0 = 7;
-			int value1 = 2;
-			int coll0 = 5;
-			int coll1 = 3;
-			
-			switch (test) {
-				case 1:
-					AA = modifyValue(A, Y,value0,coll0);
-					break;
-				case 2:
-					AA = modifyValue(A, Y, value0, coll0);
-					for(int i = 0;i<rows;i++){
-						for(int j = 0; j < cols+1;j++){
-							if(j == cols )
-								Ys[i] = (int) AA[i][j];
-							else
-								As[i][j] = AA[i][j];
-						}
-					}
-					AA = modifyValue(As,Ys,value1,coll0);
-					break;
-				case 3:
-					AA = modifyValue(A, Y, value0, coll0);
-					for(int i = 0;i<rows;i++){
-						for(int j = 0; j < cols+1;j++){
-							if(j == cols ){
-								Ys[i] = (int) AA[i][j];
-							}else{
-								As[i][j] = AA[i][j];
-							}
-						}
-
-					}
-					AA = modifyValue(As,Ys,value1,coll1);
-					break;
-			}
-			double[][] Y0 = new double[rows][1];
-			for(int i = 0; i< rows;i++){
-				Y0[i][0]= AA[i][10];
-			}
-			writeInputMatrixWithMTD("A", A, true);
-			writeInputMatrixWithMTD("B", B, true);
-			writeInputMatrixWithMTD("Y0", Y0, true);
+			programArgs = new String[]{"-args", data,
+				String.valueOf(K),String.valueOf(dp).toUpperCase(),
+				String.valueOf(VERBOSE).toUpperCase(), output("R")};
 			runTest(true, false, null, -1);
 
-			HashMap<MatrixValue.CellIndex, Double> dmlfile = readDMLMatrixFromHDFS("C");
-			double[][] ress = new double [5][20];
-			for (Entry<MatrixValue.CellIndex, Double> a : dmlfile.entrySet()) {
-				MatrixValue.CellIndex ci = a.getKey();
-				ress[ci.row-1][ci.column-1] = a.getValue();
-			}
-			for(int i = 0; i < 5; i++){
-				if(test == 1 ){
-					if(ress[i][3] == value0 && ress[i][4] ==  coll0+1 && ress[i][12] == 0 && ress[i][13] == 0){
-						System.out.print("Test passed!");
-					}
-				}else{
-					if(((ress[i][3] == value0 | ress[i][3] == value1)  && (ress[i][4] ==  coll0+1 | ress[i][4] == coll1 +1)) && ((ress[i][12] == value0 | ress[i][12] == value1)
-							&& (ress[i][13] == coll0+1 | ress[i][13] == coll1+1))){
-						System.out.print("Test passed!");
-					}
-				}
-			}
+			double[][] ret = TestUtils.convertHashMapToDoubleArray(readDMLMatrixFromHDFS("R"));
+			for(int i=0; i<K; i++)
+				TestUtils.compareMatrices(EXPECTED_TOPK[i], ret[i], 1e-2);
 		}
 		finally {
 			rtplatform = platformOld;
 		}
 	}
-	
-	private static double[][] randomizeArray(double[][]y){
-		Random rgen=new Random();
-		for(int i=0; i<y.length; i++){
-			int randomPosition=rgen.nextInt(y.length);
-			double temp=y[i][0];
-			y[i][0]=y[randomPosition][0];
-			y[randomPosition][0]=temp;
-		}
-		return y;
-	}
-
-	private static double[][] modifyValue(double[][] A, double[] Y, int value, int coll){
-		int counter = 0;
-		double nvec[][] = new double[rows][1];
-		for (int i = 0; i < rows; i++) {
-			if (A[i][coll] == value) {
-				nvec[counter][0] = Y[i];
-				counter++;
-			}
-		}
-		double[][] y = new double[counter][1];
-		for (int i = 0; i < counter; i++)
-			y[i][0] = nvec[i][0];
-
-		double[][] yy = randomizeArray(y);
-		double AA [][] = new double[rows][cols + 1];
-		counter = 0;
-
-		for(int i = 0; i<rows; i++) {
-			for(int j = 0; j < cols + 1;j++)
-				AA[i][j] = (j == cols ) ? Y[i] : A[i][j];
-			if(A[i][coll] == value) {  // this condition changes the values you choose
-				AA[i][10] = yy[counter][0];
-				counter++;
-			}
-		}
-		return AA;
-	}
 }
diff --git a/src/test/scripts/functions/builtin/data/Salaries.csv b/src/test/scripts/functions/builtin/data/Salaries.csv
new file mode 100644
index 0000000..e683235
--- /dev/null
+++ b/src/test/scripts/functions/builtin/data/Salaries.csv
@@ -0,0 +1,398 @@
+"","rank","discipline","yrs.since.phd","yrs.service","sex","salary"
+"1","Prof","B",19,18,"Male",139750
+"2","Prof","B",20,16,"Male",173200
+"3","AsstProf","B",4,3,"Male",79750
+"4","Prof","B",45,39,"Male",115000
+"5","Prof","B",40,41,"Male",141500
+"6","AssocProf","B",6,6,"Male",97000
+"7","Prof","B",30,23,"Male",175000
+"8","Prof","B",45,45,"Male",147765
+"9","Prof","B",21,20,"Male",119250
+"10","Prof","B",18,18,"Female",129000
+"11","AssocProf","B",12,8,"Male",119800
+"12","AsstProf","B",7,2,"Male",79800
+"13","AsstProf","B",1,1,"Male",77700
+"14","AsstProf","B",2,0,"Male",78000
+"15","Prof","B",20,18,"Male",104800
+"16","Prof","B",12,3,"Male",117150
+"17","Prof","B",19,20,"Male",101000
+"18","Prof","A",38,34,"Male",103450
+"19","Prof","A",37,23,"Male",124750
+"20","Prof","A",39,36,"Female",137000
+"21","Prof","A",31,26,"Male",89565
+"22","Prof","A",36,31,"Male",102580
+"23","Prof","A",34,30,"Male",93904
+"24","Prof","A",24,19,"Male",113068
+"25","AssocProf","A",13,8,"Female",74830
+"26","Prof","A",21,8,"Male",106294
+"27","Prof","A",35,23,"Male",134885
+"28","AsstProf","B",5,3,"Male",82379
+"29","AsstProf","B",11,0,"Male",77000
+"30","Prof","B",12,8,"Male",118223
+"31","Prof","B",20,4,"Male",132261
+"32","AsstProf","B",7,2,"Male",79916
+"33","Prof","B",13,9,"Male",117256
+"34","AsstProf","B",4,2,"Male",80225
+"35","AsstProf","B",4,2,"Female",80225
+"36","AsstProf","B",5,0,"Female",77000
+"37","Prof","B",22,21,"Male",155750
+"38","AsstProf","B",7,4,"Male",86373
+"39","Prof","B",41,31,"Male",125196
+"40","AssocProf","B",9,9,"Male",100938
+"41","Prof","B",23,2,"Male",146500
+"42","AssocProf","B",23,23,"Male",93418
+"43","Prof","B",40,27,"Male",101299
+"44","Prof","B",38,38,"Male",231545
+"45","Prof","B",19,19,"Male",94384
+"46","Prof","B",25,15,"Male",114778
+"47","Prof","B",40,28,"Male",98193
+"48","Prof","B",23,19,"Female",151768
+"49","Prof","B",25,25,"Female",140096
+"50","AsstProf","B",1,1,"Male",70768
+"51","Prof","B",28,28,"Male",126621
+"52","Prof","B",12,11,"Male",108875
+"53","AsstProf","B",11,3,"Female",74692
+"54","Prof","B",16,9,"Male",106639
+"55","AssocProf","B",12,11,"Male",103760
+"56","AssocProf","B",14,5,"Male",83900
+"57","Prof","B",23,21,"Male",117704
+"58","AssocProf","B",9,8,"Male",90215
+"59","AssocProf","B",10,9,"Male",100135
+"60","AsstProf","B",8,3,"Male",75044
+"61","AssocProf","B",9,8,"Male",90304
+"62","AsstProf","B",3,2,"Male",75243
+"63","Prof","B",33,31,"Male",109785
+"64","AssocProf","B",11,11,"Female",103613
+"65","AsstProf","B",4,3,"Male",68404
+"66","AssocProf","B",9,8,"Male",100522
+"67","Prof","B",22,12,"Male",101000
+"68","Prof","B",35,31,"Male",99418
+"69","Prof","B",17,17,"Female",111512
+"70","Prof","B",28,36,"Male",91412
+"71","Prof","B",17,2,"Male",126320
+"72","Prof","B",45,45,"Male",146856
+"73","Prof","B",29,19,"Male",100131
+"74","Prof","B",35,34,"Male",92391
+"75","Prof","B",28,23,"Male",113398
+"76","AsstProf","B",8,3,"Male",73266
+"77","Prof","B",17,3,"Male",150480
+"78","Prof","B",26,19,"Male",193000
+"79","AsstProf","B",3,1,"Male",86100
+"80","AsstProf","B",6,2,"Male",84240
+"81","Prof","B",43,28,"Male",150743
+"82","Prof","B",17,16,"Male",135585
+"83","Prof","B",22,20,"Male",144640
+"84","AsstProf","B",6,2,"Male",88825
+"85","Prof","B",17,18,"Female",122960
+"86","Prof","B",15,14,"Male",132825
+"87","Prof","B",37,37,"Male",152708
+"88","AsstProf","B",2,2,"Male",88400
+"89","Prof","B",25,25,"Male",172272
+"90","AssocProf","B",9,7,"Male",107008
+"91","AsstProf","B",10,5,"Female",97032
+"92","AssocProf","B",10,7,"Male",105128
+"93","AssocProf","B",10,7,"Male",105631
+"94","Prof","B",38,38,"Male",166024
+"95","Prof","B",21,20,"Male",123683
+"96","AsstProf","B",4,0,"Male",84000
+"97","AssocProf","B",17,12,"Male",95611
+"98","Prof","B",13,7,"Male",129676
+"99","Prof","B",30,14,"Male",102235
+"100","Prof","B",41,26,"Male",106689
+"101","Prof","B",42,25,"Male",133217
+"102","Prof","B",28,23,"Male",126933
+"103","Prof","B",16,5,"Male",153303
+"104","Prof","B",20,14,"Female",127512
+"105","AssocProf","A",18,10,"Male",83850
+"106","Prof","A",31,28,"Male",113543
+"107","AssocProf","A",11,8,"Male",82099
+"108","AssocProf","A",10,8,"Male",82600
+"109","AssocProf","A",15,8,"Male",81500
+"110","Prof","A",40,31,"Male",131205
+"111","Prof","A",20,16,"Male",112429
+"112","AssocProf","A",19,16,"Male",82100
+"113","AsstProf","A",3,1,"Male",72500
+"114","Prof","A",37,37,"Male",104279
+"115","Prof","A",12,0,"Female",105000
+"116","Prof","A",21,9,"Male",120806
+"117","Prof","A",30,29,"Male",148500
+"118","Prof","A",39,36,"Male",117515
+"119","AsstProf","A",4,1,"Male",72500
+"120","AsstProf","A",5,3,"Female",73500
+"121","Prof","A",14,14,"Male",115313
+"122","Prof","A",32,32,"Male",124309
+"123","Prof","A",24,22,"Male",97262
+"124","AssocProf","A",25,22,"Female",62884
+"125","Prof","A",24,22,"Male",96614
+"126","Prof","A",54,49,"Male",78162
+"127","Prof","A",28,26,"Male",155500
+"128","AsstProf","A",2,0,"Female",72500
+"129","Prof","A",32,30,"Male",113278
+"130","AsstProf","A",4,2,"Male",73000
+"131","AssocProf","A",11,9,"Male",83001
+"132","Prof","A",56,57,"Male",76840
+"133","AssocProf","A",10,8,"Female",77500
+"134","AsstProf","A",3,1,"Female",72500
+"135","Prof","A",35,25,"Male",168635
+"136","Prof","A",20,18,"Male",136000
+"137","Prof","A",16,14,"Male",108262
+"138","Prof","A",17,14,"Male",105668
+"139","AssocProf","A",10,7,"Male",73877
+"140","Prof","A",21,18,"Male",152664
+"141","AssocProf","A",14,8,"Male",100102
+"142","AssocProf","A",15,10,"Male",81500
+"143","Prof","A",19,11,"Male",106608
+"144","AsstProf","B",3,3,"Male",89942
+"145","Prof","B",27,27,"Male",112696
+"146","Prof","B",28,28,"Male",119015
+"147","AsstProf","B",4,4,"Male",92000
+"148","Prof","B",27,27,"Male",156938
+"149","Prof","B",36,26,"Female",144651
+"150","AsstProf","B",4,3,"Male",95079
+"151","Prof","B",14,12,"Male",128148
+"152","AsstProf","B",4,4,"Male",92000
+"153","Prof","B",21,9,"Male",111168
+"154","AssocProf","B",12,10,"Female",103994
+"155","AsstProf","B",4,0,"Male",92000
+"156","Prof","B",21,21,"Male",118971
+"157","AssocProf","B",12,18,"Male",113341
+"158","AsstProf","B",1,0,"Male",88000
+"159","AssocProf","B",6,6,"Male",95408
+"160","Prof","B",15,16,"Male",137167
+"161","AsstProf","B",2,2,"Male",89516
+"162","Prof","B",26,19,"Male",176500
+"163","AssocProf","B",22,7,"Male",98510
+"164","AsstProf","B",3,3,"Male",89942
+"165","AsstProf","B",1,0,"Male",88795
+"166","Prof","B",21,8,"Male",105890
+"167","Prof","B",16,16,"Male",167284
+"168","Prof","B",18,19,"Male",130664
+"169","AssocProf","B",8,6,"Male",101210
+"170","Prof","B",25,18,"Male",181257
+"171","AsstProf","B",5,5,"Male",91227
+"172","Prof","B",19,19,"Male",151575
+"173","Prof","B",37,24,"Male",93164
+"174","Prof","B",20,20,"Male",134185
+"175","AssocProf","B",17,6,"Male",105000
+"176","Prof","B",28,25,"Male",111751
+"177","AssocProf","B",10,7,"Male",95436
+"178","AssocProf","B",13,9,"Male",100944
+"179","Prof","B",27,14,"Male",147349
+"180","AsstProf","B",3,3,"Female",92000
+"181","Prof","B",11,11,"Male",142467
+"182","Prof","B",18,5,"Male",141136
+"183","AssocProf","B",8,8,"Male",100000
+"184","Prof","B",26,22,"Male",150000
+"185","Prof","B",23,23,"Male",101000
+"186","Prof","B",33,30,"Male",134000
+"187","AssocProf","B",13,10,"Female",103750
+"188","Prof","B",18,10,"Male",107500
+"189","AssocProf","B",28,28,"Male",106300
+"190","Prof","B",25,19,"Male",153750
+"191","Prof","B",22,9,"Male",180000
+"192","Prof","B",43,22,"Male",133700
+"193","Prof","B",19,18,"Male",122100
+"194","AssocProf","B",19,19,"Male",86250
+"195","AssocProf","B",48,53,"Male",90000
+"196","AssocProf","B",9,7,"Male",113600
+"197","AsstProf","B",4,4,"Male",92700
+"198","AsstProf","B",4,4,"Male",92000
+"199","Prof","B",34,33,"Male",189409
+"200","Prof","B",38,22,"Male",114500
+"201","AsstProf","B",4,4,"Male",92700
+"202","Prof","B",40,40,"Male",119700
+"203","Prof","B",28,17,"Male",160400
+"204","Prof","B",17,17,"Male",152500
+"205","Prof","B",19,5,"Male",165000
+"206","Prof","B",21,2,"Male",96545
+"207","Prof","B",35,33,"Male",162200
+"208","Prof","B",18,18,"Male",120000
+"209","AsstProf","B",7,2,"Male",91300
+"210","Prof","B",20,20,"Male",163200
+"211","AsstProf","B",4,3,"Male",91000
+"212","Prof","B",39,39,"Male",111350
+"213","Prof","B",15,7,"Male",128400
+"214","Prof","B",26,19,"Male",126200
+"215","AssocProf","B",11,1,"Male",118700
+"216","Prof","B",16,11,"Male",145350
+"217","Prof","B",15,11,"Male",146000
+"218","AssocProf","B",29,22,"Male",105350
+"219","AssocProf","B",14,7,"Female",109650
+"220","Prof","B",13,11,"Male",119500
+"221","Prof","B",21,21,"Male",170000
+"222","Prof","B",23,10,"Male",145200
+"223","AssocProf","B",13,6,"Male",107150
+"224","Prof","B",34,20,"Male",129600
+"225","Prof","A",38,35,"Male",87800
+"226","Prof","A",20,20,"Male",122400
+"227","AsstProf","A",3,1,"Male",63900
+"228","AssocProf","A",9,7,"Male",70000
+"229","Prof","A",16,11,"Male",88175
+"230","Prof","A",39,38,"Male",133900
+"231","Prof","A",29,27,"Female",91000
+"232","AssocProf","A",26,24,"Female",73300
+"233","Prof","A",38,19,"Male",148750
+"234","Prof","A",36,19,"Female",117555
+"235","AsstProf","A",8,3,"Male",69700
+"236","Prof","A",28,17,"Male",81700
+"237","Prof","A",25,25,"Male",114000
+"238","AsstProf","A",7,6,"Female",63100
+"239","Prof","A",46,40,"Male",77202
+"240","Prof","A",19,6,"Male",96200
+"241","AsstProf","A",5,3,"Male",69200
+"242","Prof","A",31,30,"Male",122875
+"243","Prof","A",38,37,"Male",102600
+"244","Prof","A",23,23,"Male",108200
+"245","Prof","A",19,23,"Male",84273
+"246","Prof","A",17,11,"Female",90450
+"247","Prof","A",30,23,"Male",91100
+"248","Prof","A",21,18,"Male",101100
+"249","Prof","A",28,23,"Male",128800
+"250","Prof","A",29,7,"Male",204000
+"251","Prof","A",39,39,"Male",109000
+"252","Prof","A",20,8,"Male",102000
+"253","Prof","A",31,12,"Male",132000
+"254","AsstProf","A",4,2,"Female",77500
+"255","Prof","A",28,7,"Female",116450
+"256","AssocProf","A",12,8,"Male",83000
+"257","Prof","A",22,22,"Male",140300
+"258","AssocProf","A",30,23,"Male",74000
+"259","AsstProf","A",9,3,"Male",73800
+"260","Prof","A",32,30,"Male",92550
+"261","AssocProf","A",41,33,"Male",88600
+"262","Prof","A",45,45,"Male",107550
+"263","Prof","A",31,26,"Male",121200
+"264","Prof","A",31,31,"Male",126000
+"265","Prof","A",37,35,"Male",99000
+"266","Prof","A",36,30,"Male",134800
+"267","Prof","A",43,43,"Male",143940
+"268","Prof","A",14,10,"Male",104350
+"269","Prof","A",47,44,"Male",89650
+"270","Prof","A",13,7,"Male",103700
+"271","Prof","A",42,40,"Male",143250
+"272","Prof","A",42,18,"Male",194800
+"273","AsstProf","A",4,1,"Male",73000
+"274","AsstProf","A",8,4,"Male",74000
+"275","AsstProf","A",8,3,"Female",78500
+"276","Prof","A",12,6,"Male",93000
+"277","Prof","A",52,48,"Male",107200
+"278","Prof","A",31,27,"Male",163200
+"279","Prof","A",24,18,"Male",107100
+"280","Prof","A",46,46,"Male",100600
+"281","Prof","A",39,38,"Male",136500
+"282","Prof","A",37,27,"Male",103600
+"283","Prof","A",51,51,"Male",57800
+"284","Prof","A",45,43,"Male",155865
+"285","AssocProf","A",8,6,"Male",88650
+"286","AssocProf","A",49,49,"Male",81800
+"287","Prof","A",28,27,"Male",115800
+"288","AsstProf","A",2,0,"Male",85000
+"289","Prof","A",29,27,"Male",150500
+"290","AsstProf","A",8,5,"Male",74000
+"291","Prof","A",33,7,"Male",174500
+"292","Prof","A",32,28,"Male",168500
+"293","Prof","A",39,9,"Male",183800
+"294","AssocProf","A",11,1,"Male",104800
+"295","Prof","A",19,7,"Male",107300
+"296","Prof","A",40,36,"Male",97150
+"297","Prof","A",18,18,"Male",126300
+"298","Prof","A",17,11,"Male",148800
+"299","Prof","A",49,43,"Male",72300
+"300","AssocProf","A",45,39,"Male",70700
+"301","Prof","A",39,36,"Male",88600
+"302","Prof","A",27,16,"Male",127100
+"303","Prof","A",28,13,"Male",170500
+"304","Prof","A",14,4,"Male",105260
+"305","Prof","A",46,44,"Male",144050
+"306","Prof","A",33,31,"Male",111350
+"307","AsstProf","A",7,4,"Male",74500
+"308","Prof","A",31,28,"Male",122500
+"309","AsstProf","A",5,0,"Male",74000
+"310","Prof","A",22,15,"Male",166800
+"311","Prof","A",20,7,"Male",92050
+"312","Prof","A",14,9,"Male",108100
+"313","Prof","A",29,19,"Male",94350
+"314","Prof","A",35,35,"Male",100351
+"315","Prof","A",22,6,"Male",146800
+"316","AsstProf","B",6,3,"Male",84716
+"317","AssocProf","B",12,9,"Female",71065
+"318","Prof","B",46,45,"Male",67559
+"319","Prof","B",16,16,"Male",134550
+"320","Prof","B",16,15,"Male",135027
+"321","Prof","B",24,23,"Male",104428
+"322","AssocProf","B",9,9,"Male",95642
+"323","AssocProf","B",13,11,"Male",126431
+"324","Prof","B",24,15,"Female",161101
+"325","Prof","B",30,31,"Male",162221
+"326","AsstProf","B",8,4,"Male",84500
+"327","Prof","B",23,15,"Male",124714
+"328","Prof","B",37,37,"Male",151650
+"329","AssocProf","B",10,10,"Male",99247
+"330","Prof","B",23,23,"Male",134778
+"331","Prof","B",49,60,"Male",192253
+"332","Prof","B",20,9,"Male",116518
+"333","Prof","B",18,10,"Female",105450
+"334","Prof","B",33,19,"Male",145098
+"335","AssocProf","B",19,6,"Female",104542
+"336","Prof","B",36,38,"Male",151445
+"337","Prof","B",35,23,"Male",98053
+"338","Prof","B",13,12,"Male",145000
+"339","Prof","B",32,25,"Male",128464
+"340","Prof","B",37,15,"Male",137317
+"341","Prof","B",13,11,"Male",106231
+"342","Prof","B",17,17,"Female",124312
+"343","Prof","B",38,38,"Male",114596
+"344","Prof","B",31,31,"Male",162150
+"345","Prof","B",32,35,"Male",150376
+"346","Prof","B",15,10,"Male",107986
+"347","Prof","B",41,27,"Male",142023
+"348","Prof","B",39,33,"Male",128250
+"349","AsstProf","B",4,3,"Male",80139
+"350","Prof","B",27,28,"Male",144309
+"351","Prof","B",56,49,"Male",186960
+"352","Prof","B",38,38,"Male",93519
+"353","Prof","B",26,27,"Male",142500
+"354","Prof","B",22,20,"Male",138000
+"355","AsstProf","B",8,1,"Male",83600
+"356","Prof","B",25,21,"Male",145028
+"357","Prof","A",49,40,"Male",88709
+"358","Prof","A",39,35,"Male",107309
+"359","Prof","A",28,14,"Female",109954
+"360","AsstProf","A",11,4,"Male",78785
+"361","Prof","A",14,11,"Male",121946
+"362","Prof","A",23,15,"Female",109646
+"363","Prof","A",30,30,"Male",138771
+"364","AssocProf","A",20,17,"Male",81285
+"365","Prof","A",43,43,"Male",205500
+"366","Prof","A",43,40,"Male",101036
+"367","Prof","A",15,10,"Male",115435
+"368","AssocProf","A",10,1,"Male",108413
+"369","Prof","A",35,30,"Male",131950
+"370","Prof","A",33,31,"Male",134690
+"371","AssocProf","A",13,8,"Male",78182
+"372","Prof","A",23,20,"Male",110515
+"373","Prof","A",12,7,"Male",109707
+"374","Prof","A",30,26,"Male",136660
+"375","Prof","A",27,19,"Male",103275
+"376","Prof","A",28,26,"Male",103649
+"377","AsstProf","A",4,1,"Male",74856
+"378","AsstProf","A",6,3,"Male",77081
+"379","Prof","A",38,38,"Male",150680
+"380","AssocProf","A",11,8,"Male",104121
+"381","AsstProf","A",8,3,"Male",75996
+"382","Prof","A",27,23,"Male",172505
+"383","AssocProf","A",8,5,"Male",86895
+"384","Prof","A",44,44,"Male",105000
+"385","Prof","A",27,21,"Male",125192
+"386","Prof","A",15,9,"Male",114330
+"387","Prof","A",29,27,"Male",139219
+"388","Prof","A",29,15,"Male",109305
+"389","Prof","A",38,36,"Male",119450
+"390","Prof","A",33,18,"Male",186023
+"391","Prof","A",40,19,"Male",166605
+"392","Prof","A",30,19,"Male",151292
+"393","Prof","A",33,30,"Male",103106
+"394","Prof","A",31,19,"Male",150564
+"395","Prof","A",42,25,"Male",101738
+"396","Prof","A",25,15,"Male",95329
+"397","AsstProf","A",8,4,"Male",81035
diff --git a/src/test/scripts/functions/builtin/slicefinder.dml b/src/test/scripts/functions/builtin/slicefinder.dml
index 8984193..442ef79 100644
--- a/src/test/scripts/functions/builtin/slicefinder.dml
+++ b/src/test/scripts/functions/builtin/slicefinder.dml
@@ -19,8 +19,24 @@
 #
 #-------------------------------------------------------------
 
-X = read($1);
-w = read($2);
-y = read($3)
-ress = slicefinder(X = X,W = w, Y = y,  k = 5, paq = 1, S = 2);
-write(ress, $4);
\ No newline at end of file
+FXY = read($1, data_type="frame", format="csv", header=TRUE);
+
+F = FXY[,1:ncol(FXY)-1];
+y = as.matrix(FXY[,ncol(FXY)]);
+
+# data preparation
+jspec= "{ ids:true, recode:[1,2,3,6],bin:[" 
+      +"{id:4, method:equi-width, numbins:14},"
+      +"{id:5, method:equi-width, numbins:12}]}"
+[X,M] = transformencode(target=F, spec=jspec);
+X = X[,2:ncol(X)]
+
+# learn model
+B = lm(X=X, y=y, verbose=FALSE);
+yhat = X %*% B;
+e = (y-yhat)^2;
+
+# call slice finding
+[TK,TKC] = slicefinder(X=X, e=e, k=$2, alpha=0.95, minSup=4, dpEval=$3, verbose=$4);
+
+write(TKC, $5)