You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@fluo.apache.org by mw...@apache.org on 2016/07/07 20:44:04 UTC

[3/3] incubator-fluo git commit: #696 - Updated integration tests to use core API instead of type layer

 #696 - Updated integration tests to use core API instead of type layer


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

Branch: refs/heads/master
Commit: b2c91b95be1b665867b5cab74e80541cb700e57a
Parents: e84b4f1
Author: Mike Walch <mw...@gmail.com>
Authored: Wed Jul 6 15:18:52 2016 -0400
Committer: Mike Walch <mw...@gmail.com>
Committed: Thu Jul 7 15:57:06 2016 -0400

----------------------------------------------------------------------
 .../org/apache/fluo/integration/BankUtil.java   |  17 +-
 .../fluo/integration/TestTransaction.java       | 103 +++++-
 .../org/apache/fluo/integration/TestUtil.java   |  42 +++
 .../fluo/integration/impl/AppConfigIT.java      |  45 ++-
 .../integration/impl/ClientExceptionIT.java     |  52 +--
 .../fluo/integration/impl/CollisionIT.java      |  63 ++--
 .../fluo/integration/impl/ColumnVisIT.java      |  13 +-
 .../apache/fluo/integration/impl/FailureIT.java | 239 +++++++-------
 .../fluo/integration/impl/FaultyConfig.java     |   6 +-
 .../apache/fluo/integration/impl/FluoIT.java    | 313 +++++++++----------
 .../impl/GarbageCollectionIteratorIT.java       |  17 +-
 .../fluo/integration/impl/NotificationGcIT.java |  25 +-
 .../fluo/integration/impl/ObserverConfigIT.java |  26 +-
 .../apache/fluo/integration/impl/OracleIT.java  |   3 -
 .../integration/impl/ParallelScannerIT.java     |  87 +++---
 .../integration/impl/SelfNotificationIT.java    |  55 ++--
 .../fluo/integration/impl/StochasticBankIT.java |  15 +-
 .../integration/impl/WeakNotificationIT.java    |  54 ++--
 .../impl/WeakNotificationOverlapIT.java         | 110 +++----
 .../apache/fluo/integration/impl/WorkerIT.java  |  83 +++--
 .../org/apache/fluo/integration/log/LogIT.java  |  50 ++-
 .../mapreduce/it/FluoFileOutputFormatIT.java    |  26 +-
 .../fluo/mapreduce/it/MutationBuilderIT.java    |  51 ++-
 23 files changed, 762 insertions(+), 733 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/b2c91b95/modules/integration/src/test/java/org/apache/fluo/integration/BankUtil.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/BankUtil.java b/modules/integration/src/test/java/org/apache/fluo/integration/BankUtil.java
index 7937c5e..37e7df2 100644
--- a/modules/integration/src/test/java/org/apache/fluo/integration/BankUtil.java
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/BankUtil.java
@@ -16,8 +16,6 @@
 package org.apache.fluo.integration;
 
 import org.apache.fluo.api.data.Column;
-import org.apache.fluo.api.types.StringEncoder;
-import org.apache.fluo.api.types.TypeLayer;
 import org.apache.fluo.core.impl.Environment;
 
 /**
@@ -25,28 +23,27 @@ import org.apache.fluo.core.impl.Environment;
  */
 public class BankUtil {
 
-  public static final TypeLayer typeLayer = new TypeLayer(new StringEncoder());
-  public static final Column BALANCE = typeLayer.bc().fam("account").qual("balance").vis();
+  public static final Column BALANCE = new Column("account", "balance");
 
   private BankUtil() {}
 
   public static void transfer(Environment env, String from, String to, int amount) throws Exception {
     TestTransaction tx = new TestTransaction(env);
 
-    int bal1 = tx.get().row(from).col(BALANCE).toInteger();
-    int bal2 = tx.get().row(to).col(BALANCE).toInteger();
+    int bal1 = Integer.parseInt(tx.gets(from, BALANCE));
+    int bal2 = Integer.parseInt(tx.gets(to, BALANCE));
 
-    tx.mutate().row(from).col(BALANCE).set(bal1 - amount);
-    tx.mutate().row(to).col(BALANCE).set(bal2 + amount);
+    tx.set(from, BALANCE, (bal1 - amount) + "");
+    tx.set(to, BALANCE, (bal2 + amount) + "");
 
     tx.done();
   }
 
   public static void setBalance(TestTransaction tx, String user, int amount) {
-    tx.mutate().row(user).col(BALANCE).set(amount);
+    tx.set(user, BALANCE, amount + "");
   }
 
   public static int getBalance(TestTransaction tx, String user) {
-    return tx.get().row(user).col(BALANCE).toInteger();
+    return Integer.parseInt(tx.gets(user, BALANCE));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/b2c91b95/modules/integration/src/test/java/org/apache/fluo/integration/TestTransaction.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/TestTransaction.java b/modules/integration/src/test/java/org/apache/fluo/integration/TestTransaction.java
index b80b980..b64cda5 100644
--- a/modules/integration/src/test/java/org/apache/fluo/integration/TestTransaction.java
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/TestTransaction.java
@@ -15,7 +15,10 @@
 
 package org.apache.fluo.integration;
 
+import java.util.Collection;
+import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -28,14 +31,14 @@ import org.apache.fluo.accumulo.iterators.NotificationIterator;
 import org.apache.fluo.accumulo.util.ColumnConstants;
 import org.apache.fluo.accumulo.util.NotificationUtil;
 import org.apache.fluo.api.client.TransactionBase;
+import org.apache.fluo.api.config.ScannerConfiguration;
 import org.apache.fluo.api.data.Bytes;
 import org.apache.fluo.api.data.Column;
 import org.apache.fluo.api.data.RowColumn;
 import org.apache.fluo.api.data.Span;
+import org.apache.fluo.api.exceptions.AlreadySetException;
 import org.apache.fluo.api.exceptions.CommitException;
-import org.apache.fluo.api.types.StringEncoder;
-import org.apache.fluo.api.types.TypeLayer;
-import org.apache.fluo.api.types.TypedTransactionBase;
+import org.apache.fluo.api.iterator.RowIterator;
 import org.apache.fluo.core.exceptions.AlreadyAcknowledgedException;
 import org.apache.fluo.core.impl.Environment;
 import org.apache.fluo.core.impl.Notification;
@@ -48,7 +51,7 @@ import org.apache.fluo.core.util.ByteUtil;
 import org.apache.fluo.core.util.SpanUtil;
 import org.apache.hadoop.io.Text;
 
-public class TestTransaction extends TypedTransactionBase implements TransactionBase {
+public class TestTransaction implements TransactionBase {
 
   private TransactionImpl tx;
   private Environment env;
@@ -78,18 +81,15 @@ public class TestTransaction extends TypedTransactionBase implements Transaction
     throw new RuntimeException("No notification found");
   }
 
-  @SuppressWarnings("resource")
   public TestTransaction(Environment env, TransactorNode transactor) {
-    this(new TransactionImpl(env).setTransactor(transactor), new StringEncoder(), env);
+    this(new TransactionImpl(env).setTransactor(transactor), env);
   }
 
   public TestTransaction(Environment env) {
-    this(new TransactionImpl(env), new StringEncoder(), env);
+    this(new TransactionImpl(env), env);
   }
 
-  private TestTransaction(TransactionImpl transactionImpl, StringEncoder stringEncoder,
-      Environment env) {
-    super(transactionImpl, stringEncoder, new TypeLayer(stringEncoder));
+  private TestTransaction(TransactionImpl transactionImpl, Environment env) {
     this.tx = transactionImpl;
     this.env = env;
   }
@@ -99,8 +99,7 @@ public class TestTransaction extends TypedTransactionBase implements Transaction
   }
 
   public TestTransaction(Environment env, String trow, Column tcol, long notificationTS) {
-    this(new TransactionImpl(env, new Notification(Bytes.of(trow), tcol, notificationTS)),
-        new StringEncoder(), env);
+    this(new TransactionImpl(env, new Notification(Bytes.of(trow), tcol, notificationTS)), env);
   }
 
   /**
@@ -154,4 +153,84 @@ public class TestTransaction extends TypedTransactionBase implements Transaction
   public TxStats getStats() {
     return tx.getStats();
   }
+
+  @Override
+  public void delete(Bytes row, Column col) {
+    tx.delete(row, col);
+  }
+
+  @Override
+  public void delete(String row, Column col) {
+    tx.delete(row, col);
+  }
+
+  @Override
+  public void set(Bytes row, Column col, Bytes value) throws AlreadySetException {
+    tx.set(row, col, value);
+  }
+
+  @Override
+  public void set(String row, Column col, String value) throws AlreadySetException {
+    tx.set(row, col, value);
+  }
+
+  @Override
+  public void setWeakNotification(Bytes row, Column col) {
+    tx.setWeakNotification(row, col);
+  }
+
+  @Override
+  public void setWeakNotification(String row, Column col) {
+    tx.setWeakNotification(row, col);
+  }
+
+  @Override
+  public Bytes get(Bytes row, Column column) {
+    return tx.get(row, column);
+  }
+
+  @Override
+  public Map<Column, Bytes> get(Bytes row, Set<Column> columns) {
+    return tx.get(row, columns);
+  }
+
+  @Override
+  public Map<Bytes, Map<Column, Bytes>> get(Collection<Bytes> rows, Set<Column> columns) {
+    return tx.get(rows, columns);
+  }
+
+  @Override
+  public Map<Bytes, Map<Column, Bytes>> get(Collection<RowColumn> rowColumns) {
+    return tx.get(rowColumns);
+  }
+
+  @Override
+  public RowIterator get(ScannerConfiguration config) {
+    return tx.get(config);
+  }
+
+  @Override
+  public Map<String, Map<Column, String>> gets(Collection<RowColumn> rowColumns) {
+    return tx.gets(rowColumns);
+  }
+
+  @Override
+  public Map<String, Map<Column, String>> gets(Collection<String> rows, Set<Column> columns) {
+    return tx.gets(rows, columns);
+  }
+
+  @Override
+  public String gets(String row, Column column) {
+    return tx.gets(row, column);
+  }
+
+  @Override
+  public Map<Column, String> gets(String row, Set<Column> columns) {
+    return tx.gets(row, columns);
+  }
+
+  @Override
+  public long getStartTimestamp() {
+    return tx.getStartTimestamp();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/b2c91b95/modules/integration/src/test/java/org/apache/fluo/integration/TestUtil.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/TestUtil.java b/modules/integration/src/test/java/org/apache/fluo/integration/TestUtil.java
new file mode 100644
index 0000000..fe9ee27
--- /dev/null
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/TestUtil.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.fluo.integration;
+
+import org.apache.fluo.api.client.SnapshotBase;
+import org.apache.fluo.api.client.TransactionBase;
+import org.apache.fluo.api.data.Column;
+
+public class TestUtil {
+
+  private TestUtil() {}
+
+  public static void increment(TransactionBase tx, String row, Column col, int val) {
+    int prev = 0;
+    String prevStr = tx.gets(row, col);
+    if (prevStr != null) {
+      prev = Integer.parseInt(prevStr);
+    }
+    tx.set(row, col, prev + val + "");
+  }
+
+  public static int getOrDefault(SnapshotBase snap, String row, Column col, int defaultVal) {
+    String val = snap.gets(row, col);
+    if (val == null) {
+      return defaultVal;
+    }
+    return Integer.parseInt(val);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/b2c91b95/modules/integration/src/test/java/org/apache/fluo/integration/impl/AppConfigIT.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/impl/AppConfigIT.java b/modules/integration/src/test/java/org/apache/fluo/integration/impl/AppConfigIT.java
index fedacbf..74f40e4 100644
--- a/modules/integration/src/test/java/org/apache/fluo/integration/impl/AppConfigIT.java
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/impl/AppConfigIT.java
@@ -21,17 +21,15 @@ import java.util.List;
 import org.apache.fluo.api.client.FluoAdmin;
 import org.apache.fluo.api.client.FluoClient;
 import org.apache.fluo.api.client.FluoFactory;
+import org.apache.fluo.api.client.Loader;
 import org.apache.fluo.api.client.LoaderExecutor;
+import org.apache.fluo.api.client.Snapshot;
+import org.apache.fluo.api.client.TransactionBase;
 import org.apache.fluo.api.config.ObserverConfiguration;
 import org.apache.fluo.api.config.SimpleConfiguration;
 import org.apache.fluo.api.data.Bytes;
 import org.apache.fluo.api.data.Column;
-import org.apache.fluo.api.types.StringEncoder;
-import org.apache.fluo.api.types.TypeLayer;
-import org.apache.fluo.api.types.TypedLoader;
-import org.apache.fluo.api.types.TypedObserver;
-import org.apache.fluo.api.types.TypedSnapshot;
-import org.apache.fluo.api.types.TypedTransactionBase;
+import org.apache.fluo.api.observer.AbstractObserver;
 import org.apache.fluo.integration.ITBaseMini;
 import org.junit.Assert;
 import org.junit.Test;
@@ -77,7 +75,7 @@ public class AppConfigIT extends ITBaseMini {
 
   }
 
-  public static class TestLoader extends TypedLoader {
+  private static class TestLoader implements Loader {
 
     private String row;
     private int data;
@@ -88,16 +86,15 @@ public class AppConfigIT extends ITBaseMini {
     }
 
     @Override
-    public void load(TypedTransactionBase tx, Context context) throws Exception {
+    public void load(TransactionBase tx, Context context) throws Exception {
       int limit = context.getAppConfiguration().getInt("myapp.sizeLimit");
       if (data < limit) {
-        tx.mutate().row(row).fam("data").qual("foo").set(data);
+        tx.set(row, new Column("data", "foo"), Integer.toString(data));
       }
     }
-
   }
 
-  public static class TestObserver extends TypedObserver {
+  public static class TestObserver extends AbstractObserver {
 
     private int limit;
 
@@ -112,13 +109,12 @@ public class AppConfigIT extends ITBaseMini {
     }
 
     @Override
-    public void process(TypedTransactionBase tx, Bytes row, Column col) {
-      int d = tx.get().row(row).col(col).toInteger();
+    public void process(TransactionBase tx, Bytes row, Column col) throws Exception {
+      int d = Integer.parseInt(tx.gets(row.toString(), col));
       if (2 * d < limit) {
-        tx.mutate().row(row).fam("data").qual("bar").set(2 * d);
+        tx.set(row.toString(), new Column("data", "bar"), Integer.toString(2 * d));
       }
     }
-
   }
 
   @Test
@@ -130,21 +126,18 @@ public class AppConfigIT extends ITBaseMini {
       le.execute(new TestLoader("r3", 60000));
     }
 
-    TypeLayer tl = new TypeLayer(new StringEncoder());
-
-    try (TypedSnapshot snapshot = tl.wrap(client.newSnapshot())) {
-      Assert.assertEquals(3, snapshot.get().row("r1").fam("data").qual("foo").toInteger(0));
-      Assert.assertEquals(30000, snapshot.get().row("r2").fam("data").qual("foo").toInteger(0));
-      Assert.assertEquals(0, snapshot.get().row("r3").fam("data").qual("foo").toInteger(0));
+    try (Snapshot snapshot = client.newSnapshot()) {
+      Assert.assertEquals("3", snapshot.gets("r1", new Column("data", "foo")));
+      Assert.assertEquals("30000", snapshot.gets("r2", new Column("data", "foo")));
+      Assert.assertNull(snapshot.gets("r3", new Column("data", "foo")));
     }
 
     miniFluo.waitForObservers();
 
-    try (TypedSnapshot snapshot = tl.wrap(client.newSnapshot())) {
-      Assert.assertEquals(6, snapshot.get().row("r1").fam("data").qual("bar").toInteger(0));
-      Assert.assertEquals(0, snapshot.get().row("r2").fam("data").qual("bar").toInteger(0));
-      Assert.assertEquals(0, snapshot.get().row("r3").fam("data").qual("bar").toInteger(0));
+    try (Snapshot snapshot = client.newSnapshot()) {
+      Assert.assertEquals("6", snapshot.gets("r1", new Column("data", "bar")));
+      Assert.assertNull(snapshot.gets("r2", new Column("data", "bar")));
+      Assert.assertNull(snapshot.gets("r3", new Column("data", "bar")));
     }
-
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/b2c91b95/modules/integration/src/test/java/org/apache/fluo/integration/impl/ClientExceptionIT.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/impl/ClientExceptionIT.java b/modules/integration/src/test/java/org/apache/fluo/integration/impl/ClientExceptionIT.java
index 30419ab..9807101 100644
--- a/modules/integration/src/test/java/org/apache/fluo/integration/impl/ClientExceptionIT.java
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/impl/ClientExceptionIT.java
@@ -19,9 +19,6 @@ import org.apache.fluo.api.client.Transaction;
 import org.apache.fluo.api.data.Bytes;
 import org.apache.fluo.api.data.Column;
 import org.apache.fluo.api.exceptions.AlreadySetException;
-import org.apache.fluo.api.types.StringEncoder;
-import org.apache.fluo.api.types.TypeLayer;
-import org.apache.fluo.api.types.TypedTransaction;
 import org.apache.fluo.integration.ITBaseMini;
 import org.junit.Assert;
 import org.junit.Test;
@@ -31,8 +28,6 @@ import org.junit.Test;
  */
 public class ClientExceptionIT extends ITBaseMini {
 
-  static TypeLayer tl = new TypeLayer(new StringEncoder());
-
   @Test
   public void testAlreadySetException() {
 
@@ -42,54 +37,23 @@ public class ClientExceptionIT extends ITBaseMini {
       tx.set(Bytes.of("row"), new Column("c1"), Bytes.of("val2"));
       Assert.fail("exception not thrown");
     } catch (AlreadySetException e) {
+      // do nothing
     }
 
     try (Transaction tx = client.newTransaction()) {
-      tx.delete(Bytes.of("row"), new Column("c1"));
-      tx.delete(Bytes.of("row"), new Column("c1"));
-      Assert.fail("exception not thrown");
-    } catch (AlreadySetException e) {
-    }
-
-    // test typed transactions
-    // setting integer
-    try (TypedTransaction tx = tl.wrap(client.newTransaction())) {
-      tx.mutate().row("r1").col(new Column("c1")).set("a");
-      tx.mutate().row("r1").col(new Column("c1")).set(6);
-      Assert.fail("exception not thrown");
-    } catch (AlreadySetException e) {
-    }
-
-    // test set setting empty twice
-    try (TypedTransaction tx = tl.wrap(client.newTransaction())) {
-      tx.mutate().row("r1").col(new Column("c1")).set();
-      tx.mutate().row("r1").col(new Column("c1")).set();
+      tx.set("row", new Column("c2"), "a");
+      tx.set("row", new Column("c2"), "b");
       Assert.fail("exception not thrown");
     } catch (AlreadySetException e) {
+      // do nothing
     }
 
-    // test boolean and same value
-    try (TypedTransaction tx = tl.wrap(client.newTransaction())) {
-      tx.mutate().row("r1").col(new Column("c1")).set(true);
-      tx.mutate().row("r1").col(new Column("c1")).set(true);
-      Assert.fail("exception not thrown");
-    } catch (AlreadySetException e) {
-    }
-
-    // test string
-    try (TypedTransaction tx = tl.wrap(client.newTransaction())) {
-      tx.mutate().row("r1").col(new Column("c1")).set("a");
-      tx.mutate().row("r1").col(new Column("c1")).set("b");
-      Assert.fail("exception not thrown");
-    } catch (AlreadySetException e) {
-    }
-
-    // test two deletes
-    try (TypedTransaction tx = tl.wrap(client.newTransaction())) {
-      tx.mutate().row("r1").col(new Column("c1")).delete();
-      tx.mutate().row("r1").col(new Column("c1")).delete();
+    try (Transaction tx = client.newTransaction()) {
+      tx.delete(Bytes.of("row"), new Column("c1"));
+      tx.delete(Bytes.of("row"), new Column("c1"));
       Assert.fail("exception not thrown");
     } catch (AlreadySetException e) {
+      // do nothing
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/b2c91b95/modules/integration/src/test/java/org/apache/fluo/integration/impl/CollisionIT.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/impl/CollisionIT.java b/modules/integration/src/test/java/org/apache/fluo/integration/impl/CollisionIT.java
index 543150c..4af93d1 100644
--- a/modules/integration/src/test/java/org/apache/fluo/integration/impl/CollisionIT.java
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/impl/CollisionIT.java
@@ -15,7 +15,7 @@
 
 package org.apache.fluo.integration.impl;
 
-import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map.Entry;
@@ -27,20 +27,20 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.fluo.accumulo.util.ColumnConstants;
 import org.apache.fluo.accumulo.util.ZookeeperUtil;
+import org.apache.fluo.api.client.Loader;
 import org.apache.fluo.api.client.LoaderExecutor;
+import org.apache.fluo.api.client.Snapshot;
+import org.apache.fluo.api.client.TransactionBase;
 import org.apache.fluo.api.config.FluoConfiguration;
 import org.apache.fluo.api.config.ObserverConfiguration;
 import org.apache.fluo.api.data.Bytes;
 import org.apache.fluo.api.data.Column;
-import org.apache.fluo.api.types.StringEncoder;
-import org.apache.fluo.api.types.TypeLayer;
-import org.apache.fluo.api.types.TypedLoader;
-import org.apache.fluo.api.types.TypedObserver;
-import org.apache.fluo.api.types.TypedSnapshot;
-import org.apache.fluo.api.types.TypedTransactionBase;
+import org.apache.fluo.api.observer.AbstractObserver;
+import org.apache.fluo.api.observer.Observer;
 import org.apache.fluo.core.impl.FluoConfigurationImpl;
 import org.apache.fluo.core.util.UtilWaitThread;
 import org.apache.fluo.integration.ITBaseMini;
+import org.apache.fluo.integration.TestUtil;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -56,9 +56,12 @@ import org.junit.Test;
  *
  */
 public class CollisionIT extends ITBaseMini {
-  static TypeLayer typeLayer = new TypeLayer(new StringEncoder());
 
-  static class NumLoader extends TypedLoader {
+  private static final Column STAT_TOTAL = new Column("stat", "total");
+  private static final Column STAT_CHANGED = new Column("stat", "changed");
+  private static final Column STAT_PROCESSED = new Column("stat", "processed");
+
+  private static class NumLoader implements Loader {
 
     int num;
 
@@ -67,33 +70,33 @@ public class CollisionIT extends ITBaseMini {
     }
 
     @Override
-    public void load(TypedTransactionBase tx, Context context) throws Exception {
-      tx.mutate().row(num).fam("stat").qual("total").increment(1);
-      tx.mutate().row(num).fam("stat").qual("changed").weaklyNotify();
+    public void load(TransactionBase tx, Context context) throws Exception {
+      TestUtil.increment(tx, num + "", STAT_TOTAL, 1);
+      tx.setWeakNotification(num + "", STAT_CHANGED);
     }
   }
 
-  public static class TotalObserver extends TypedObserver {
+  public static class TotalObserver extends AbstractObserver {
 
     @Override
-    public ObservedColumn getObservedColumn() {
-      return new ObservedColumn(typeLayer.bc().fam("stat").qual("changed").vis(),
-          NotificationType.WEAK);
+    public Observer.ObservedColumn getObservedColumn() {
+      return new Observer.ObservedColumn(STAT_CHANGED, NotificationType.WEAK);
     }
 
     @Override
-    public void process(TypedTransactionBase tx, Bytes row, Column col) {
-      int total = tx.get().row(row).fam("stat").qual("total").toInteger();
-      int processed = tx.get().row(row).fam("stat").qual("processed").toInteger(0);
+    public void process(TransactionBase tx, Bytes rowBytes, Column col) throws Exception {
+      String row = rowBytes.toString();
+      int total = Integer.parseInt(tx.gets(row, STAT_TOTAL));
+      int processed = TestUtil.getOrDefault(tx, row, STAT_PROCESSED, 0);
 
-      tx.mutate().row(row).fam("stat").qual("processed").set(total);
-      tx.mutate().row("all").fam("stat").qual("total").increment(total - processed);
+      tx.set(row, STAT_PROCESSED, total + "");
+      TestUtil.increment(tx, "all", STAT_TOTAL, total - processed);
     }
   }
 
   @Override
   protected List<ObserverConfiguration> getObservers() {
-    return Arrays.asList(new ObserverConfiguration(TotalObserver.class.getName()));
+    return Collections.singletonList(new ObserverConfiguration(TotalObserver.class.getName()));
   }
 
   @Override
@@ -127,16 +130,20 @@ public class CollisionIT extends ITBaseMini {
 
     miniFluo.waitForObservers();
 
-    try (TypedSnapshot snapshot = typeLayer.wrap(client.newSnapshot())) {
+    try (Snapshot snapshot = client.newSnapshot()) {
 
       for (int i = 0; i < expectedCounts.length; i++) {
-        Assert.assertEquals(expectedCounts[i], snapshot.get().row(i).fam("stat").qual("total")
-            .toInteger(-1));
-        Assert.assertEquals(expectedCounts[i], snapshot.get().row(i).fam("stat").qual("processed")
-            .toInteger(-1));
+        String total = snapshot.gets(i + "", STAT_TOTAL);
+        Assert.assertNotNull(total);
+        Assert.assertEquals(expectedCounts[i], Integer.parseInt(total));
+        String processed = snapshot.gets(i + "", STAT_PROCESSED);
+        Assert.assertNotNull(processed);
+        Assert.assertEquals(expectedCounts[i], Integer.parseInt(processed));
       }
 
-      Assert.assertEquals(1000, snapshot.get().row("all").fam("stat").qual("total").toInteger(-1));
+      String allTotal = snapshot.gets("all", STAT_TOTAL);
+      Assert.assertNotNull(allTotal);
+      Assert.assertEquals(1000, Integer.parseInt(allTotal));
     }
 
     long oldestTS = ZookeeperUtil.getGcTimestamp(config.getAppZookeepers());

http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/b2c91b95/modules/integration/src/test/java/org/apache/fluo/integration/impl/ColumnVisIT.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/impl/ColumnVisIT.java b/modules/integration/src/test/java/org/apache/fluo/integration/impl/ColumnVisIT.java
index 27646c9..2f94b26 100644
--- a/modules/integration/src/test/java/org/apache/fluo/integration/impl/ColumnVisIT.java
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/impl/ColumnVisIT.java
@@ -17,6 +17,7 @@ package org.apache.fluo.integration.impl;
 
 import java.util.Arrays;
 
+import com.google.common.collect.Sets;
 import org.apache.fluo.api.data.Column;
 import org.apache.fluo.integration.ITBaseImpl;
 import org.apache.fluo.integration.TestTransaction;
@@ -29,7 +30,7 @@ public class ColumnVisIT extends ITBaseImpl {
     TestTransaction tx1 = new TestTransaction(env);
 
     // expect set w/ bad col vis to fail fast
-    tx1.mutate().row("r").fam("f").qual("q").vis("A&").set("v");
+    tx1.set("r", new Column("f", "q", "A&"), "v");
   }
 
   @Test(expected = Exception.class)
@@ -37,7 +38,7 @@ public class ColumnVisIT extends ITBaseImpl {
     TestTransaction tx1 = new TestTransaction(env);
 
     // expect delete w/ bad col vis to fail fast
-    tx1.mutate().row("r").fam("f").qual("q").vis("A&").delete();
+    tx1.delete("r", new Column("f", "q", "A&"));
   }
 
   @Test(expected = Exception.class)
@@ -45,7 +46,7 @@ public class ColumnVisIT extends ITBaseImpl {
     TestTransaction tx1 = new TestTransaction(env);
 
     // expect weaknotify w/ bad col vis to fail fast
-    tx1.mutate().row("r").fam("f").qual("q").vis("A&").weaklyNotify();
+    tx1.setWeakNotification("r", new Column("f", "q", "A&"));
   }
 
   @Test(expected = Exception.class)
@@ -53,7 +54,7 @@ public class ColumnVisIT extends ITBaseImpl {
     TestTransaction tx1 = new TestTransaction(env);
 
     // expect get w/ bad col vis to fail fast
-    tx1.get().row("r").fam("f").qual("q").vis("A&").toString();
+    tx1.gets("r", new Column("f", "q", "A&"));
   }
 
   @Test(expected = Exception.class)
@@ -64,7 +65,7 @@ public class ColumnVisIT extends ITBaseImpl {
     Column col2 = new Column("f", "q", "C|");
 
     // expect get cols w/ bad col vis to fail fast
-    tx1.get().row("r").columns(col1, col2).size();
+    tx1.gets("r", Sets.newHashSet(col1, col2)).size();
   }
 
   @Test(expected = Exception.class)
@@ -75,6 +76,6 @@ public class ColumnVisIT extends ITBaseImpl {
     Column col2 = new Column("f", "q", "C|");
 
     // expect get rows cols w/ bad col vis to fail fast
-    tx1.get().rowsString(Arrays.asList("r1", "r2")).columns(col1, col2).toStringMap().size();
+    tx1.gets(Arrays.asList("r1", "r2"), Sets.newHashSet(col1, col2)).size();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/b2c91b95/modules/integration/src/test/java/org/apache/fluo/integration/impl/FailureIT.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/impl/FailureIT.java b/modules/integration/src/test/java/org/apache/fluo/integration/impl/FailureIT.java
index d0b9986..38a857e 100644
--- a/modules/integration/src/test/java/org/apache/fluo/integration/impl/FailureIT.java
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/impl/FailureIT.java
@@ -38,8 +38,6 @@ import org.apache.fluo.api.data.Column;
 import org.apache.fluo.api.exceptions.CommitException;
 import org.apache.fluo.api.exceptions.FluoException;
 import org.apache.fluo.api.observer.AbstractObserver;
-import org.apache.fluo.api.types.StringEncoder;
-import org.apache.fluo.api.types.TypeLayer;
 import org.apache.fluo.core.exceptions.AlreadyAcknowledgedException;
 import org.apache.fluo.core.exceptions.StaleScanException;
 import org.apache.fluo.core.impl.Notification;
@@ -50,6 +48,7 @@ import org.apache.fluo.core.oracle.Stamp;
 import org.apache.fluo.integration.BankUtil;
 import org.apache.fluo.integration.ITBaseImpl;
 import org.apache.fluo.integration.TestTransaction;
+import org.apache.fluo.integration.TestUtil;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
@@ -62,8 +61,6 @@ public class FailureIT extends ITBaseImpl {
   @Rule
   public ExpectedException exception = ExpectedException.none();
 
-  static TypeLayer typeLayer = new TypeLayer(new StringEncoder());
-
   public static class NullObserver extends AbstractObserver {
 
     @Override
@@ -71,8 +68,7 @@ public class FailureIT extends ITBaseImpl {
 
     @Override
     public ObservedColumn getObservedColumn() {
-      return new ObservedColumn(typeLayer.bc().fam("attr").qual("lastupdate").vis(),
-          NotificationType.STRONG);
+      return new ObservedColumn(new Column("attr", "lastupdate"), NotificationType.STRONG);
     }
   }
 
@@ -93,18 +89,19 @@ public class FailureIT extends ITBaseImpl {
     testRollbackMany(false);
   }
 
-  public void testRollbackMany(boolean killTransactor) throws Exception {
+  private void testRollbackMany(boolean killTransactor) throws Exception {
 
     // test writing lots of columns that need to be rolled back
 
-    Column col1 = typeLayer.bc().fam("fam1").qual("q1").vis();
-    Column col2 = typeLayer.bc().fam("fam1").qual("q2").vis();
+    Column col1 = new Column("fam1", "q1");
+    Column col2 = new Column("fam1", "q2");
 
     TestTransaction tx = new TestTransaction(env);
 
     for (int r = 0; r < 10; r++) {
-      tx.mutate().row(r + "").col(col1).set("0" + r + "0");
-      tx.mutate().row(r + "").col(col2).set("0" + r + "1");
+      String row = Integer.toString(r);
+      tx.set(row, col1, "0" + r + "0");
+      tx.set(row, col2, "0" + r + "1");
     }
 
     tx.done();
@@ -113,8 +110,8 @@ public class FailureIT extends ITBaseImpl {
     TestTransaction tx2 = new TestTransaction(env, t2);
 
     for (int r = 0; r < 10; r++) {
-      tx2.mutate().row(r + "").col(col1).set("1" + r + "0");
-      tx2.mutate().row(r + "").col(col2).set("1" + r + "1");
+      tx2.set(r + "", col1, "1" + r + "0");
+      tx2.set(r + "", col2, "1" + r + "1");
     }
 
     CommitData cd = tx2.createCommitData();
@@ -126,8 +123,8 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx3 = new TestTransaction(env);
     for (int r = 0; r < 10; r++) {
-      Assert.assertEquals("0" + r + "0", tx3.get().row(r + "").col(col1).toString());
-      Assert.assertEquals("0" + r + "1", tx3.get().row(r + "").col(col2).toString());
+      Assert.assertEquals("0" + r + "0", tx3.gets(r + "", col1));
+      Assert.assertEquals("0" + r + "1", tx3.gets(r + "", col2));
     }
 
     if (killTransactor) {
@@ -142,8 +139,8 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx4 = new TestTransaction(env);
     for (int r = 0; r < 10; r++) {
-      Assert.assertEquals("0" + r + "0", tx4.get().row(r + "").col(col1).toString());
-      Assert.assertEquals("0" + r + "1", tx4.get().row(r + "").col(col2).toString());
+      Assert.assertEquals("0" + r + "0", tx4.gets(r + "", col1));
+      Assert.assertEquals("0" + r + "1", tx4.gets(r + "", col2));
     }
   }
 
@@ -157,17 +154,17 @@ public class FailureIT extends ITBaseImpl {
     testRollforwardMany(false);
   }
 
-  public void testRollforwardMany(boolean killTransactor) throws Exception {
+  private void testRollforwardMany(boolean killTransactor) throws Exception {
     // test writing lots of columns that need to be rolled forward
 
-    Column col1 = typeLayer.bc().fam("fam1").qual("q1").vis();
-    Column col2 = typeLayer.bc().fam("fam1").qual("q2").vis();
+    Column col1 = new Column("fam1", "q1");
+    Column col2 = new Column("fam1", "q2");
 
     TestTransaction tx = new TestTransaction(env);
 
     for (int r = 0; r < 10; r++) {
-      tx.mutate().row(r + "").col(col1).set("0" + r + "0");
-      tx.mutate().row(r + "").col(col2).set("0" + r + "1");
+      tx.set(r + "", col1, "0" + r + "0");
+      tx.set(r + "", col2, "0" + r + "1");
     }
 
     tx.done();
@@ -176,8 +173,8 @@ public class FailureIT extends ITBaseImpl {
     TestTransaction tx2 = new TestTransaction(env, t2);
 
     for (int r = 0; r < 10; r++) {
-      tx2.mutate().row(r + "").col(col1).set("1" + r + "0");
-      tx2.mutate().row(r + "").col(col2).set("1" + r + "1");
+      tx2.set(r + "", col1, "1" + r + "0");
+      tx2.set(r + "", col2, "1" + r + "1");
     }
 
     CommitData cd = tx2.createCommitData();
@@ -191,16 +188,16 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx3 = new TestTransaction(env);
     for (int r = 0; r < 10; r++) {
-      Assert.assertEquals("1" + r + "0", tx3.get().row(r + "").col(col1).toString());
-      Assert.assertEquals("1" + r + "1", tx3.get().row(r + "").col(col2).toString());
+      Assert.assertEquals("1" + r + "0", tx3.gets(r + "", col1));
+      Assert.assertEquals("1" + r + "1", tx3.gets(r + "", col2));
     }
 
     tx2.finishCommit(cd, commitTs);
 
     TestTransaction tx4 = new TestTransaction(env);
     for (int r = 0; r < 10; r++) {
-      Assert.assertEquals("1" + r + "0", tx4.get().row(r + "").col(col1).toString());
-      Assert.assertEquals("1" + r + "1", tx4.get().row(r + "").col(col2).toString());
+      Assert.assertEquals("1" + r + "0", tx4.gets(r + "", col1));
+      Assert.assertEquals("1" + r + "1", tx4.gets(r + "", col2));
     }
 
     if (!killTransactor) {
@@ -214,19 +211,19 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx = new TestTransaction(env);
 
-    tx.mutate().row("bob").col(BALANCE).set(10);
-    tx.mutate().row("joe").col(BALANCE).set(20);
-    tx.mutate().row("jill").col(BALANCE).set(60);
+    tx.set("bob", BALANCE, "10");
+    tx.set("joe", BALANCE, "20");
+    tx.set("jill", BALANCE, "60");
 
     tx.done();
 
     TestTransaction tx2 = new TestTransaction(env);
 
-    int bal1 = tx2.get().row("bob").col(BALANCE).toInteger(0);
-    int bal2 = tx2.get().row("joe").col(BALANCE).toInteger(0);
+    int bal1 = Integer.parseInt(tx2.gets("bob", BALANCE));
+    int bal2 = Integer.parseInt(tx2.gets("joe", BALANCE));
 
-    tx2.mutate().row("bob").col(BALANCE).set(bal1 - 7);
-    tx2.mutate().row("joe").col(BALANCE).set(bal2 + 7);
+    tx2.set("bob", BALANCE, (bal1 - 7) + "");
+    tx2.set("joe", BALANCE, (bal2 + 7) + "");
 
     // get locks
     CommitData cd = tx2.createCommitData();
@@ -246,9 +243,9 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx4 = new TestTransaction(env);
 
-    Assert.assertEquals(bobBal, tx4.get().row("bob").col(BALANCE).toInteger(0));
-    Assert.assertEquals(joeBal, tx4.get().row("joe").col(BALANCE).toInteger(0));
-    Assert.assertEquals(67, tx4.get().row("jill").col(BALANCE).toInteger(0));
+    Assert.assertEquals(bobBal + "", tx4.gets("bob", BALANCE));
+    Assert.assertEquals(joeBal + "", tx4.gets("joe", BALANCE));
+    Assert.assertEquals("67", tx4.gets("jill", BALANCE));
 
     Stamp commitTs = env.getSharedResources().getOracleClient().getStamp();
     Assert.assertFalse(tx2.commitPrimaryColumn(cd, commitTs));
@@ -259,9 +256,9 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx6 = new TestTransaction(env);
 
-    Assert.assertEquals(bobBal, tx6.get().row("bob").col(BALANCE).toInteger(0));
-    Assert.assertEquals(joeBal, tx6.get().row("joe").col(BALANCE).toInteger(0));
-    Assert.assertEquals(67, tx6.get().row("jill").col(BALANCE).toInteger(0));
+    Assert.assertEquals(bobBal + "", tx6.gets("bob", BALANCE));
+    Assert.assertEquals(joeBal + "", tx6.gets("joe", BALANCE));
+    Assert.assertEquals("67", tx6.gets("jill", BALANCE));
   }
 
   @Test
@@ -279,19 +276,19 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx = new TestTransaction(env);
 
-    tx.mutate().row("bob").col(BALANCE).set(10);
-    tx.mutate().row("joe").col(BALANCE).set(20);
-    tx.mutate().row("jill").col(BALANCE).set(60);
+    tx.set("bob", BALANCE, "10");
+    tx.set("joe", BALANCE, "20");
+    tx.set("jill", BALANCE, "60");
 
     tx.done();
 
     TestTransaction tx2 = new TestTransaction(env, t1);
 
-    int bal1 = tx2.get().row("bob").col(BALANCE).toInteger(0);
-    int bal2 = tx2.get().row("joe").col(BALANCE).toInteger(0);
+    int bal1 = Integer.parseInt(tx2.gets("bob", BALANCE));
+    int bal2 = Integer.parseInt(tx2.gets("joe", BALANCE));
 
-    tx2.mutate().row("bob").col(BALANCE).set(bal1 - 7);
-    tx2.mutate().row("joe").col(BALANCE).set(bal2 + 7);
+    tx2.set("bob", BALANCE, (bal1 - 7) + "");
+    tx2.set("joe", BALANCE, (bal2 + 7) + "");
 
     CommitData cd = tx2.createCommitData();
     Assert.assertTrue(tx2.preCommit(cd));
@@ -338,19 +335,19 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx = new TestTransaction(env);
 
-    tx.mutate().row("bob").col(BALANCE).set(10);
-    tx.mutate().row("joe").col(BALANCE).set(20);
-    tx.mutate().row("jill").col(BALANCE).set(60);
+    tx.set("bob", BALANCE, "10");
+    tx.set("joe", BALANCE, "20");
+    tx.set("jill", BALANCE, "60");
 
     tx.done();
 
     TestTransaction tx2 = new TestTransaction(env);
 
-    int bal1 = tx2.get().row("bob").col(BALANCE).toInteger(0);
-    int bal2 = tx2.get().row("joe").col(BALANCE).toInteger(0);
+    int bal1 = Integer.parseInt(tx2.gets("bob", BALANCE));
+    int bal2 = Integer.parseInt(tx2.gets("joe", BALANCE));
 
-    tx2.mutate().row("bob").col(BALANCE).set(bal1 - 7);
-    tx2.mutate().row("joe").col(BALANCE).set(bal2 + 7);
+    tx2.set("bob", BALANCE, (bal1 - 7) + "");
+    tx2.set("joe", BALANCE, (bal2 + 7) + "");
 
     // get locks
     CommitData cd = tx2.createCommitData();
@@ -371,17 +368,17 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx4 = new TestTransaction(env);
 
-    Assert.assertEquals(bobBal, tx4.get().row("bob").col(BALANCE).toInteger(0));
-    Assert.assertEquals(joeBal, tx4.get().row("joe").col(BALANCE).toInteger(0));
-    Assert.assertEquals(62, tx4.get().row("jill").col(BALANCE).toInteger(0));
+    Assert.assertEquals(bobBal + "", tx4.gets("bob", BALANCE));
+    Assert.assertEquals(joeBal + "", tx4.gets("joe", BALANCE));
+    Assert.assertEquals("62", tx4.gets("jill", BALANCE));
 
     tx2.finishCommit(cd, commitTs);
 
     TestTransaction tx5 = new TestTransaction(env);
 
-    Assert.assertEquals(bobBal, tx5.get().row("bob").col(BALANCE).toInteger(0));
-    Assert.assertEquals(joeBal, tx5.get().row("joe").col(BALANCE).toInteger(0));
-    Assert.assertEquals(62, tx5.get().row("jill").col(BALANCE).toInteger(0));
+    Assert.assertEquals(bobBal + "", tx5.gets("bob", BALANCE));
+    Assert.assertEquals(joeBal + "", tx5.gets("joe", BALANCE));
+    Assert.assertEquals("62", tx5.gets("jill", BALANCE));
   }
 
   @Test
@@ -390,22 +387,25 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx = new TestTransaction(env);
 
-    tx.mutate().row("url0000").fam("attr").qual("lastupdate").set(3);
-    tx.mutate().row("url0000").fam("doc").qual("content").set("abc def");
+    final Column lastUpdate = new Column("attr", "lastupdate");
+    final Column docContent = new Column("doc", "content");
+    final Column docUrl = new Column("doc", "url");
+
+    tx.set("url0000", lastUpdate, "3");
+    tx.set("url0000", docContent, "abc def");
 
     tx.done();
 
-    TestTransaction tx2 =
-        new TestTransaction(env, "url0000", typeLayer.bc().fam("attr").qual("lastupdate").vis());
-    tx2.mutate().row("idx:abc").fam("doc").qual("url").set("url0000");
-    tx2.mutate().row("idx:def").fam("doc").qual("url").set("url0000");
+    TestTransaction tx2 = new TestTransaction(env, "url0000", lastUpdate);
+    tx2.set("idx:abc", docUrl, "url0000");
+    tx2.set("idx:def", docUrl, "url0000");
     CommitData cd = tx2.createCommitData();
     tx2.preCommit(cd);
 
     TestTransaction tx3 = new TestTransaction(env);
-    Assert.assertNull(tx3.get().row("idx:abc").fam("doc").qual("url").toString());
-    Assert.assertNull(tx3.get().row("idx:def").fam("doc").qual("url").toString());
-    Assert.assertEquals(3, tx3.get().row("url0000").fam("attr").qual("lastupdate").toInteger(0));
+    Assert.assertNull(tx3.gets("idx:abc", docUrl));
+    Assert.assertNull(tx3.gets("idx:def", docUrl));
+    Assert.assertEquals("3", tx3.gets("url0000", lastUpdate));
 
     Scanner scanner = env.getConnector().createScanner(env.getTable(), Authorizations.EMPTY);
     Notification.configureScanner(scanner);
@@ -413,10 +413,9 @@ public class FailureIT extends ITBaseImpl {
     Assert.assertTrue(iter.hasNext());
     Assert.assertEquals("url0000", iter.next().getKey().getRow().toString());
 
-    TestTransaction tx5 =
-        new TestTransaction(env, "url0000", typeLayer.bc().fam("attr").qual("lastupdate").vis());
-    tx5.mutate().row("idx:abc").fam("doc").qual("url").set("url0000");
-    tx5.mutate().row("idx:def").fam("doc").qual("url").set("url0000");
+    TestTransaction tx5 = new TestTransaction(env, "url0000", lastUpdate);
+    tx5.set("idx:abc", docUrl, "url0000");
+    tx5.set("idx:def", docUrl, "url0000");
     cd = tx5.createCommitData();
     Assert.assertTrue(tx5.preCommit(cd));
     Stamp commitTs = env.getSharedResources().getOracleClient().getStamp();
@@ -424,25 +423,25 @@ public class FailureIT extends ITBaseImpl {
 
     // should roll tx5 forward
     TestTransaction tx6 = new TestTransaction(env);
-    Assert.assertEquals(3, tx6.get().row("url0000").fam("attr").qual("lastupdate").toInteger(0));
-    Assert.assertEquals("url0000", tx6.get().row("idx:abc").fam("doc").qual("url").toString());
-    Assert.assertEquals("url0000", tx6.get().row("idx:def").fam("doc").qual("url").toString());
+    Assert.assertEquals("3", tx6.gets("url0000", lastUpdate));
+    Assert.assertEquals("url0000", tx6.gets("idx:abc", docUrl));
+    Assert.assertEquals("url0000", tx6.gets("idx:def", docUrl));
 
     iter = scanner.iterator();
     Assert.assertTrue(iter.hasNext());
 
     // TODO is tx4 start before tx5, then this test will not work because AlreadyAck is not thrown
     // for overlapping.. CommitException is thrown
-    TestTransaction tx4 =
-        new TestTransaction(env, "url0000", typeLayer.bc().fam("attr").qual("lastupdate").vis());
-    tx4.mutate().row("idx:abc").fam("doc").qual("url").set("url0000");
-    tx4.mutate().row("idx:def").fam("doc").qual("url").set("url0000");
+    TestTransaction tx4 = new TestTransaction(env, "url0000", lastUpdate);
+    tx4.set("idx:abc", docUrl, "url0000");
+    tx4.set("idx:def", docUrl, "url0000");
 
     try {
       // should not go through if tx5 is properly rolled forward
       tx4.commit();
       Assert.fail();
     } catch (AlreadyAcknowledgedException aae) {
+      // do nothing
     }
 
     // commit above should schedule async delete of notification
@@ -456,14 +455,14 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx = new TestTransaction(env);
 
-    tx.mutate().row("bob").col(BALANCE).set(10);
-    tx.mutate().row("joe").col(BALANCE).set(20);
-    tx.mutate().row("jill").col(BALANCE).set(60);
+    tx.set("bob", BALANCE, "10");
+    tx.set("joe", BALANCE, "20");
+    tx.set("jill", BALANCE, "60");
 
     tx.done();
 
     TestTransaction tx2 = new TestTransaction(env);
-    Assert.assertEquals(10, tx2.get().row("bob").col(BALANCE).toInteger(0));
+    Assert.assertEquals("10", tx2.gets("bob", BALANCE));
 
     BankUtil.transfer(env, "joe", "jill", 1);
     BankUtil.transfer(env, "joe", "bob", 1);
@@ -472,16 +471,16 @@ public class FailureIT extends ITBaseImpl {
 
     conn.tableOperations().flush(table, null, null, true);
 
-    Assert.assertEquals(20, tx2.get().row("joe").col(BALANCE).toInteger(0));
+    Assert.assertEquals("20", tx2.gets("joe", BALANCE));
 
     // Stale scan should not occur due to oldest active timestamp tracking in Zookeeper
     tx2.close();
 
     TestTransaction tx3 = new TestTransaction(env);
 
-    Assert.assertEquals(9, tx3.get().row("bob").col(BALANCE).toInteger(0));
-    Assert.assertEquals(22, tx3.get().row("joe").col(BALANCE).toInteger(0));
-    Assert.assertEquals(59, tx3.get().row("jill").col(BALANCE).toInteger(0));
+    Assert.assertEquals("9", tx3.gets("bob", BALANCE));
+    Assert.assertEquals("22", tx3.gets("joe", BALANCE));
+    Assert.assertEquals("59", tx3.gets("jill", BALANCE));
   }
 
   @Test(timeout = 60000)
@@ -489,18 +488,18 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx = new TestTransaction(env);
 
-    tx.mutate().row("bob").col(BALANCE).set(10);
-    tx.mutate().row("joe").col(BALANCE).set(20);
-    tx.mutate().row("jill").col(BALANCE).set(60);
-    tx.mutate().row("john").col(BALANCE).set(3);
+    tx.set("bob", BALANCE, "10");
+    tx.set("joe", BALANCE, "20");
+    tx.set("jill", BALANCE, "60");
+    tx.set("john", BALANCE, "3");
 
     tx.done();
 
     TestTransaction tx2 = new TestTransaction(env);
-    Assert.assertEquals(10, tx2.get().row("bob").col(BALANCE).toInteger(0));
+    Assert.assertEquals("10", tx2.gets("bob", BALANCE));
 
     TestTransaction tx3 = new TestTransaction(env);
-    tx3.get().row("john").col(BALANCE).toInteger(0);
+    tx3.gets("john", BALANCE);
 
     BankUtil.transfer(env, "joe", "jill", 1);
     BankUtil.transfer(env, "joe", "bob", 1);
@@ -527,17 +526,17 @@ public class FailureIT extends ITBaseImpl {
     conn.tableOperations().flush(table, null, null, true);
 
     // this data should have been GCed, but the problem is not detected here
-    Assert.assertNull(tx2.get().row("joe").col(BALANCE).toInteger());
+    Assert.assertNull(tx2.gets("joe", BALANCE));
 
     try {
       // closing should detect the stale scan
       tx2.close();
       Assert.assertFalse(true);
     } catch (StaleScanException sse) {
-
+      // do nothing
     }
 
-    tx3.mutate().row("john").col(BALANCE).set(5l);
+    tx3.set("john", BALANCE, "5");
 
     try {
       tx3.commit();
@@ -549,10 +548,10 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx4 = new TestTransaction(env);
 
-    Assert.assertEquals(9, tx4.get().row("bob").col(BALANCE).toInteger(0));
-    Assert.assertEquals(22, tx4.get().row("joe").col(BALANCE).toInteger(0));
-    Assert.assertEquals(59, tx4.get().row("jill").col(BALANCE).toInteger(0));
-    Assert.assertEquals(3, tx4.get().row("john").col(BALANCE).toInteger(0));
+    Assert.assertEquals("9", tx4.gets("bob", BALANCE));
+    Assert.assertEquals("22", tx4.gets("joe", BALANCE));
+    Assert.assertEquals("59", tx4.gets("jill", BALANCE));
+    Assert.assertEquals("3", tx4.gets("john", BALANCE));
   }
 
   @Test
@@ -560,9 +559,9 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx1 = new TestTransaction(env);
 
-    tx1.mutate().row("bob").col(BALANCE).set(10);
-    tx1.mutate().row("joe").col(BALANCE).set(20);
-    tx1.mutate().row("jill").col(BALANCE).set(60);
+    tx1.set("bob", BALANCE, "10");
+    tx1.set("joe", BALANCE, "20");
+    tx1.set("jill", BALANCE, "60");
 
     CommitData cd = tx1.createCommitData();
     Assert.assertTrue(tx1.preCommit(cd));
@@ -570,23 +569,23 @@ public class FailureIT extends ITBaseImpl {
     while (true) {
       TestTransaction tx2 = new TestTransaction(env);
 
-      tx2.mutate().row("bob").col(BALANCE).set(11);
-      tx2.mutate().row("jill").col(BALANCE).set(61);
+      tx2.set("bob", BALANCE, "11");
+      tx2.set("jill", BALANCE, "61");
 
       // tx1 should be rolled back even in case where columns tx1 locked are not read by tx2
       try {
         tx2.commit();
         break;
       } catch (CommitException ce) {
-
+        // do nothing
       }
     }
 
     TestTransaction tx4 = new TestTransaction(env);
 
-    Assert.assertEquals(11, tx4.get().row("bob").col(BALANCE).toInteger(0));
-    Assert.assertNull(tx4.get().row("joe").col(BALANCE).toInteger());
-    Assert.assertEquals(61, tx4.get().row("jill").col(BALANCE).toInteger(0));
+    Assert.assertEquals("11", tx4.gets("bob", BALANCE));
+    Assert.assertNull(tx4.gets("joe", BALANCE));
+    Assert.assertEquals("61", tx4.gets("jill", BALANCE));
   }
 
   @Test
@@ -596,9 +595,9 @@ public class FailureIT extends ITBaseImpl {
 
     TestTransaction tx1 = new TestTransaction(env);
 
-    tx1.mutate().row("bob").col(BALANCE).set(10);
-    tx1.mutate().row("joe").col(BALANCE).set(20);
-    tx1.mutate().row("jill").col(BALANCE).set(60);
+    tx1.set("bob", BALANCE, "10");
+    tx1.set("joe", BALANCE, "20");
+    tx1.set("jill", BALANCE, "60");
 
     tx1.done();
 
@@ -606,13 +605,12 @@ public class FailureIT extends ITBaseImpl {
     TestTransaction tx2 = new TestTransaction(env, "jill", BALANCE, 1);
 
     TestTransaction tx3 = new TestTransaction(env);
-    tx3.mutate().row("bob").col(BALANCE).increment(5);
-    tx3.mutate().row("joe").col(BALANCE).increment(-5);
+    TestUtil.increment(tx3, "bob", BALANCE, 5);
+    TestUtil.increment(tx3, "joe", BALANCE, -5);
     tx3.done();
 
-    tx2.mutate().row("bob").col(BALANCE).increment(5);
-    tx2.mutate().row("jill").col(BALANCE).increment(-5);
-
+    TestUtil.increment(tx2, "bob", BALANCE, 5);
+    TestUtil.increment(tx2, "jill", BALANCE, -5);
 
     // should be able to successfully lock the primary column jill... but then should fail to lock
     // bob and have to rollback
@@ -620,10 +618,9 @@ public class FailureIT extends ITBaseImpl {
       tx2.commit();
       Assert.fail("Expected commit exception");
     } catch (CommitException ce) {
-
+      // do nothing
     }
 
-
     boolean sawExpected = wasRolledBackPrimary(tx2.getStartTimestamp(), "jill");
 
     Assert.assertTrue(sawExpected);

http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/b2c91b95/modules/integration/src/test/java/org/apache/fluo/integration/impl/FaultyConfig.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/impl/FaultyConfig.java b/modules/integration/src/test/java/org/apache/fluo/integration/impl/FaultyConfig.java
index 7486b5a..4da6b08 100644
--- a/modules/integration/src/test/java/org/apache/fluo/integration/impl/FaultyConfig.java
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/impl/FaultyConfig.java
@@ -43,13 +43,11 @@ public class FaultyConfig extends Environment {
     private Random rand;
     private double wp;
 
-    public FaultyConditionalWriter(ConditionalWriter cw, double unknownProbability,
-        double writeProbability) {
+    FaultyConditionalWriter(ConditionalWriter cw, double unknownProbability, double writeProbability) {
       this.cw = cw;
       this.up = unknownProbability;
       this.wp = writeProbability;
       this.rand = new Random();
-
     }
 
     @Override
@@ -93,7 +91,7 @@ public class FaultyConfig extends Environment {
   private double up;
   private double wp;
 
-  public FaultyConfig(Environment env, double up, double wp) throws Exception {
+  FaultyConfig(Environment env, double up, double wp) throws Exception {
     super(env);
     this.up = up;
     this.wp = wp;

http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/b2c91b95/modules/integration/src/test/java/org/apache/fluo/integration/impl/FluoIT.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/impl/FluoIT.java b/modules/integration/src/test/java/org/apache/fluo/integration/impl/FluoIT.java
index 661ebf2..321374b 100644
--- a/modules/integration/src/test/java/org/apache/fluo/integration/impl/FluoIT.java
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/impl/FluoIT.java
@@ -28,6 +28,7 @@ import org.apache.fluo.api.client.FluoAdmin;
 import org.apache.fluo.api.client.FluoClient;
 import org.apache.fluo.api.client.FluoFactory;
 import org.apache.fluo.api.client.Snapshot;
+import org.apache.fluo.api.client.TransactionBase;
 import org.apache.fluo.api.config.FluoConfiguration;
 import org.apache.fluo.api.config.ObserverConfiguration;
 import org.apache.fluo.api.config.ScannerConfiguration;
@@ -37,33 +38,30 @@ import org.apache.fluo.api.data.Span;
 import org.apache.fluo.api.exceptions.CommitException;
 import org.apache.fluo.api.iterator.ColumnIterator;
 import org.apache.fluo.api.iterator.RowIterator;
-import org.apache.fluo.api.types.StringEncoder;
-import org.apache.fluo.api.types.TypeLayer;
-import org.apache.fluo.api.types.TypedObserver;
-import org.apache.fluo.api.types.TypedTransactionBase;
+import org.apache.fluo.api.observer.AbstractObserver;
 import org.apache.fluo.core.exceptions.AlreadyAcknowledgedException;
 import org.apache.fluo.core.impl.Environment;
 import org.apache.fluo.core.impl.TransactionImpl.CommitData;
 import org.apache.fluo.core.oracle.Stamp;
 import org.apache.fluo.integration.ITBaseImpl;
 import org.apache.fluo.integration.TestTransaction;
+import org.apache.fluo.integration.TestUtil;
 import org.junit.Assert;
 import org.junit.Test;
 
-public class FluoIT extends ITBaseImpl {
+import static org.apache.fluo.integration.BankUtil.BALANCE;
 
-  static TypeLayer typeLayer = new TypeLayer(new StringEncoder());
+public class FluoIT extends ITBaseImpl {
 
-  public static class BalanceObserver extends TypedObserver {
+  public static class BalanceObserver extends AbstractObserver {
 
     @Override
     public ObservedColumn getObservedColumn() {
-      return new ObservedColumn(typeLayer.bc().fam("account").qual("balance").vis(),
-          NotificationType.STRONG);
+      return new ObservedColumn(BALANCE, NotificationType.STRONG);
     }
 
     @Override
-    public void process(TypedTransactionBase tx, Bytes row, Column col) {
+    public void process(TransactionBase tx, Bytes row, Column col) {
       Assert.fail();
     }
   }
@@ -102,42 +100,36 @@ public class FluoIT extends ITBaseImpl {
 
     TestTransaction tx = new TestTransaction(env);
 
-    Column balanceCol = typeLayer.bc().fam("account").qual("balance").vis();
-
-    tx.mutate().row("bob").col(balanceCol).set(10);
-    tx.mutate().row("joe").col(balanceCol).set(20);
-    tx.mutate().row("jill").col(balanceCol).set(60);
+    tx.set("bob", BALANCE, "10");
+    tx.set("joe", BALANCE, "20");
+    tx.set("jill", BALANCE, "60");
 
     tx.done();
 
     tx = new TestTransaction(env);
 
-    int bal1 = tx.get().row("bob").col(balanceCol).toInteger(0);
-    int bal2 = tx.get().row("joe").col(balanceCol).toInteger(0);
-    Assert.assertEquals(10, bal1);
-    Assert.assertEquals(20, bal2);
+    Assert.assertEquals("10", tx.gets("bob", BALANCE));
+    Assert.assertEquals("20", tx.gets("joe", BALANCE));
 
-    tx.mutate().row("bob").col(balanceCol).set(bal1 - 5);
-    tx.mutate().row("joe").col(balanceCol).set(bal2 + 5);
+    TestUtil.increment(tx, "bob", BALANCE, -5);
+    TestUtil.increment(tx, "joe", BALANCE, 5);
 
     TestTransaction tx2 = new TestTransaction(env);
 
-    int bal3 = tx2.get().row("bob").col(balanceCol).toInteger(0);
-    int bal4 = tx2.get().row("jill").col(balanceCol).toInteger(0);
-    Assert.assertEquals(10, bal3);
-    Assert.assertEquals(60, bal4);
+    Assert.assertEquals("10", tx2.gets("bob", BALANCE));
+    Assert.assertEquals("60", tx2.gets("jill", BALANCE));
 
-    tx2.mutate().row("bob").col(balanceCol).set(bal3 - 5);
-    tx2.mutate().row("jill").col(balanceCol).set(bal4 + 5);
+    TestUtil.increment(tx2, "bob", BALANCE, -5);
+    TestUtil.increment(tx2, "jill", BALANCE, 5);
 
     tx2.done();
     assertCommitFails(tx);
 
     TestTransaction tx3 = new TestTransaction(env);
 
-    Assert.assertEquals(5, tx3.get().row("bob").col(balanceCol).toInteger(0));
-    Assert.assertEquals(20, tx3.get().row("joe").col(balanceCol).toInteger(0));
-    Assert.assertEquals(65, tx3.get().row("jill").col(balanceCol).toInteger(0));
+    Assert.assertEquals("5", tx3.gets("bob", BALANCE));
+    Assert.assertEquals("20", tx3.gets("joe", BALANCE));
+    Assert.assertEquals("65", tx3.gets("jill", BALANCE));
     tx3.done();
   }
 
@@ -168,12 +160,10 @@ public class FluoIT extends ITBaseImpl {
 
     TestTransaction tx = new TestTransaction(env);
 
-    Column balanceCol = typeLayer.bc().fam("account").qual("balance").vis();
-
-    tx.mutate().row("bob").col(balanceCol).set(10);
-    tx.mutate().row("joe").col(balanceCol).set(20);
-    tx.mutate().row("jill").col(balanceCol).set(60);
-    tx.mutate().row("jane").col(balanceCol).set(0);
+    tx.set("bob", BALANCE, "10");
+    tx.set("joe", BALANCE, "20");
+    tx.set("jill", BALANCE, "60");
+    tx.set("jane", BALANCE, "0");
 
     tx.done();
 
@@ -181,51 +171,45 @@ public class FluoIT extends ITBaseImpl {
 
     TestTransaction tx2 = new TestTransaction(env);
 
-    tx2.mutate().row("bob").col(balanceCol).set(tx2.get().row("bob").col(balanceCol).toLong() - 5);
-    tx2.mutate().row("joe").col(balanceCol).set(tx2.get().row("joe").col(balanceCol).toLong() - 5);
-    tx2.mutate().row("jill").col(balanceCol)
-        .set(tx2.get().row("jill").col(balanceCol).toLong() + 10);
+    TestUtil.increment(tx2, "bob", BALANCE, -5);
+    TestUtil.increment(tx2, "joe", BALANCE, -5);
+    TestUtil.increment(tx2, "jill", BALANCE, 10);
 
-    long bal1 = tx1.get().row("bob").col(balanceCol).toLong();
+    Assert.assertEquals("10", tx1.gets("bob", BALANCE));
 
     tx2.done();
 
     TestTransaction txd = new TestTransaction(env);
-    txd.mutate().row("jane").col(balanceCol).delete();
+    txd.delete("jane", BALANCE);
     txd.done();
 
-    long bal2 = tx1.get().row("joe").col(balanceCol).toLong();
-    long bal3 = tx1.get().row("jill").col(balanceCol).toLong();
-    long bal4 = tx1.get().row("jane").col(balanceCol).toLong();
+    Assert.assertEquals("20", tx1.gets("joe", BALANCE));
+    Assert.assertEquals("60", tx1.gets("jill", BALANCE));
+    Assert.assertEquals("0", tx1.gets("jane", BALANCE));
 
-    Assert.assertEquals(10l, bal1);
-    Assert.assertEquals(20l, bal2);
-    Assert.assertEquals(60l, bal3);
-    Assert.assertEquals(0l, bal4);
-
-    tx1.mutate().row("bob").col(balanceCol).set(bal1 - 5);
-    tx1.mutate().row("joe").col(balanceCol).set(bal2 + 5);
+    tx1.set("bob", BALANCE, "5");
+    tx1.set("joe", BALANCE, "25");
 
     assertCommitFails(tx1);
 
     TestTransaction tx3 = new TestTransaction(env);
 
     TestTransaction tx4 = new TestTransaction(env);
-    tx4.mutate().row("jane").col(balanceCol).set(3);
+    tx4.set("jane", BALANCE, "3");
     tx4.done();
 
-    Assert.assertEquals(5l, tx3.get().row("bob").col(balanceCol).toLong(0));
-    Assert.assertEquals(15l, tx3.get().row("joe").col(balanceCol).toLong(0));
-    Assert.assertEquals(70l, tx3.get().row("jill").col(balanceCol).toLong(0));
-    Assert.assertNull(tx3.get().row("jane").col(balanceCol).toLong());
+    Assert.assertEquals("5", tx3.gets("bob", BALANCE));
+    Assert.assertEquals("15", tx3.gets("joe", BALANCE));
+    Assert.assertEquals("70", tx3.gets("jill", BALANCE));
+    Assert.assertNull(tx3.gets("jane", BALANCE));
     tx3.done();
 
     TestTransaction tx5 = new TestTransaction(env);
 
-    Assert.assertEquals(5l, tx5.get().row("bob").col(balanceCol).toLong(0));
-    Assert.assertEquals(15l, tx5.get().row("joe").col(balanceCol).toLong(0));
-    Assert.assertEquals(70l, tx5.get().row("jill").col(balanceCol).toLong(0));
-    Assert.assertEquals(3l, tx5.get().row("jane").col(balanceCol).toLong(0));
+    Assert.assertEquals("5", tx5.gets("bob", BALANCE));
+    Assert.assertEquals("15", tx5.gets("joe", BALANCE));
+    Assert.assertEquals("70", tx5.gets("jill", BALANCE));
+    Assert.assertEquals("3", tx5.gets("jane", BALANCE));
     tx5.done();
   }
 
@@ -235,45 +219,43 @@ public class FluoIT extends ITBaseImpl {
 
     TestTransaction tx = new TestTransaction(env);
 
-    Column balanceCol = typeLayer.bc().fam("account").qual("balance").vis();
-
-    tx.mutate().row("bob").col(balanceCol).set(10);
-    tx.mutate().row("joe").col(balanceCol).set(20);
-    tx.mutate().row("jill").col(balanceCol).set(60);
+    tx.set("bob", BALANCE, "10");
+    tx.set("joe", BALANCE, "20");
+    tx.set("jill", BALANCE, "60");
 
     tx.done();
 
-    TestTransaction tx1 = new TestTransaction(env, "joe", balanceCol);
-    tx1.get().row("joe").col(balanceCol);
-    tx1.mutate().row("jill").col(balanceCol).set(61);
+    TestTransaction tx1 = new TestTransaction(env, "joe", BALANCE);
+    tx1.gets("joe", BALANCE);
+    tx1.set("jill", BALANCE, "61");
 
-    TestTransaction tx2 = new TestTransaction(env, "joe", balanceCol);
-    tx2.get().row("joe").col(balanceCol);
-    tx2.mutate().row("bob").col(balanceCol).set(11);
+    TestTransaction tx2 = new TestTransaction(env, "joe", BALANCE);
+    tx2.gets("joe", BALANCE);
+    tx2.set("bob", BALANCE, "11");
 
     tx1.done();
     assertAAck(tx2);
 
     TestTransaction tx3 = new TestTransaction(env);
 
-    Assert.assertEquals(10, tx3.get().row("bob").col(balanceCol).toInteger(0));
-    Assert.assertEquals(20, tx3.get().row("joe").col(balanceCol).toInteger(0));
-    Assert.assertEquals(61, tx3.get().row("jill").col(balanceCol).toInteger(0));
+    Assert.assertEquals("10", tx3.gets("bob", BALANCE));
+    Assert.assertEquals("20", tx3.gets("joe", BALANCE));
+    Assert.assertEquals("61", tx3.gets("jill", BALANCE));
 
     // update joe, so it can be acknowledged again
-    tx3.mutate().row("joe").col(balanceCol).set(21);
+    tx3.set("joe", BALANCE, "21");
 
     tx3.done();
 
-    TestTransaction tx4 = new TestTransaction(env, "joe", balanceCol);
-    tx4.get().row("joe").col(balanceCol);
-    tx4.mutate().row("jill").col(balanceCol).set(62);
+    TestTransaction tx4 = new TestTransaction(env, "joe", BALANCE);
+    tx4.gets("joe", BALANCE);
+    tx4.set("jill", BALANCE, "62");
 
-    TestTransaction tx5 = new TestTransaction(env, "joe", balanceCol);
-    tx5.get().row("joe").col(balanceCol);
-    tx5.mutate().row("bob").col(balanceCol).set(11);
+    TestTransaction tx5 = new TestTransaction(env, "joe", BALANCE);
+    tx5.gets("joe", BALANCE);
+    tx5.set("bob", BALANCE, "11");
 
-    TestTransaction tx7 = new TestTransaction(env, "joe", balanceCol);
+    TestTransaction tx7 = new TestTransaction(env, "joe", BALANCE);
 
     // make the 2nd transaction to start commit 1st
     tx5.done();
@@ -281,22 +263,22 @@ public class FluoIT extends ITBaseImpl {
 
     TestTransaction tx6 = new TestTransaction(env);
 
-    Assert.assertEquals(11, tx6.get().row("bob").col(balanceCol).toInteger(0));
-    Assert.assertEquals(21, tx6.get().row("joe").col(balanceCol).toInteger(0));
-    Assert.assertEquals(61, tx6.get().row("jill").col(balanceCol).toInteger(0));
+    Assert.assertEquals("11", tx6.gets("bob", BALANCE));
+    Assert.assertEquals("21", tx6.gets("joe", BALANCE));
+    Assert.assertEquals("61", tx6.gets("jill", BALANCE));
     tx6.done();
 
-    tx7.get().row("joe").col(balanceCol);
-    tx7.mutate().row("bob").col(balanceCol).set(15);
-    tx7.mutate().row("jill").col(balanceCol).set(60);
+    tx7.gets("joe", BALANCE);
+    tx7.set("bob", BALANCE, "15");
+    tx7.set("jill", BALANCE, "60");
 
     assertAAck(tx7);
 
     TestTransaction tx8 = new TestTransaction(env);
 
-    Assert.assertEquals(11, tx8.get().row("bob").col(balanceCol).toInteger(0));
-    Assert.assertEquals(21, tx8.get().row("joe").col(balanceCol).toInteger(0));
-    Assert.assertEquals(61, tx8.get().row("jill").col(balanceCol).toInteger(0));
+    Assert.assertEquals("11", tx8.gets("bob", BALANCE));
+    Assert.assertEquals("21", tx8.gets("joe", BALANCE));
+    Assert.assertEquals("61", tx8.gets("jill", BALANCE));
     tx8.done();
   }
 
@@ -304,27 +286,26 @@ public class FluoIT extends ITBaseImpl {
   public void testAck2() throws Exception {
     TestTransaction tx = new TestTransaction(env);
 
-    Column balanceCol = typeLayer.bc().fam("account").qual("balance").vis();
-    Column addrCol = typeLayer.bc().fam("account").qual("addr").vis();
+    Column addrCol = new Column("account", "addr");
 
-    tx.mutate().row("bob").col(balanceCol).set(10);
-    tx.mutate().row("joe").col(balanceCol).set(20);
-    tx.mutate().row("jill").col(balanceCol).set(60);
+    tx.set("bob", BALANCE, "10");
+    tx.set("joe", BALANCE, "20");
+    tx.set("jill", BALANCE, "60");
 
     tx.done();
 
-    TestTransaction tx1 = new TestTransaction(env, "bob", balanceCol);
-    TestTransaction tx2 = new TestTransaction(env, "bob", balanceCol);
-    TestTransaction tx3 = new TestTransaction(env, "bob", balanceCol);
+    TestTransaction tx1 = new TestTransaction(env, "bob", BALANCE);
+    TestTransaction tx2 = new TestTransaction(env, "bob", BALANCE);
+    TestTransaction tx3 = new TestTransaction(env, "bob", BALANCE);
 
-    tx1.get().row("bob").col(balanceCol).toInteger();
-    tx2.get().row("bob").col(balanceCol).toInteger();
+    tx1.gets("bob", BALANCE);
+    tx2.gets("bob", BALANCE);
 
-    tx1.get().row("bob").col(addrCol).toInteger();
-    tx2.get().row("bob").col(addrCol).toInteger();
+    tx1.gets("bob", addrCol);
+    tx2.gets("bob", addrCol);
 
-    tx1.mutate().row("bob").col(addrCol).set("1 loop pl");
-    tx2.mutate().row("bob").col(addrCol).set("1 loop pl");
+    tx1.set("bob", addrCol, "1 loop pl");
+    tx2.set("bob", addrCol, "1 loop pl");
 
     // this test overlaps the commits of two transactions w/ the same trigger
 
@@ -338,7 +319,7 @@ public class FluoIT extends ITBaseImpl {
     tx1.finishCommit(cd, commitTs);
     tx1.close();
 
-    tx3.mutate().row("bob").col(addrCol).set("2 loop pl");
+    tx3.set("bob", addrCol, "2 loop pl");
     assertAAck(tx3);
   }
 
@@ -346,22 +327,20 @@ public class FluoIT extends ITBaseImpl {
   public void testAck3() throws Exception {
     TestTransaction tx = new TestTransaction(env);
 
-    Column balanceCol = typeLayer.bc().fam("account").qual("balance").vis();
-
-    tx.mutate().row("bob").col(balanceCol).set(10);
-    tx.mutate().row("joe").col(balanceCol).set(20);
-    tx.mutate().row("jill").col(balanceCol).set(60);
+    tx.set("bob", BALANCE, "10");
+    tx.set("joe", BALANCE, "20");
+    tx.set("jill", BALANCE, "60");
 
     tx.done();
 
-    long notTS1 = TestTransaction.getNotificationTS(env, "bob", balanceCol);
+    long notTS1 = TestTransaction.getNotificationTS(env, "bob", BALANCE);
 
     // this transaction should create a second notification
     TestTransaction tx1 = new TestTransaction(env);
-    tx1.mutate().row("bob").col(balanceCol).set(11);
+    tx1.set("bob", BALANCE, "11");
     tx1.done();
 
-    long notTS2 = TestTransaction.getNotificationTS(env, "bob", balanceCol);
+    long notTS2 = TestTransaction.getNotificationTS(env, "bob", BALANCE);
 
     Assert.assertTrue(notTS1 < notTS2);
 
@@ -369,17 +348,17 @@ public class FluoIT extends ITBaseImpl {
     // should execute
     // google paper calls this message collapsing
 
-    TestTransaction tx3 = new TestTransaction(env, "bob", balanceCol, notTS1);
+    TestTransaction tx3 = new TestTransaction(env, "bob", BALANCE, notTS1);
 
-    TestTransaction tx2 = new TestTransaction(env, "bob", balanceCol, notTS1);
-    Assert.assertEquals(11, tx2.get().row("bob").col(balanceCol).toInteger(0));
+    TestTransaction tx2 = new TestTransaction(env, "bob", BALANCE, notTS1);
+    Assert.assertEquals("11", tx2.gets("bob", BALANCE));
     tx2.done();
 
-    Assert.assertEquals(11, tx3.get().row("bob").col(balanceCol).toInteger(0));
+    Assert.assertEquals("11", tx3.gets("bob", BALANCE));
     assertAAck(tx3);
 
-    TestTransaction tx4 = new TestTransaction(env, "bob", balanceCol, notTS2);
-    Assert.assertEquals(11, tx4.get().row("bob").col(balanceCol).toInteger(0));
+    TestTransaction tx4 = new TestTransaction(env, "bob", BALANCE, notTS2);
+    Assert.assertEquals("11", tx4.gets("bob", BALANCE));
     assertAAck(tx4);
   }
 
@@ -390,31 +369,29 @@ public class FluoIT extends ITBaseImpl {
 
     TestTransaction tx = new TestTransaction(env);
 
-    Column balanceCol = typeLayer.bc().fam("account").qual("balance").vis();
-
-    tx.mutate().row("bob").col(balanceCol).set(10);
-    tx.mutate().row("joe").col(balanceCol).set(20);
-    tx.mutate().row("jill").col(balanceCol).set(60);
+    tx.set("bob", BALANCE, "10");
+    tx.set("joe", BALANCE, "20");
+    tx.set("jill", BALANCE, "60");
 
     tx.done();
 
-    TestTransaction tx2 = new TestTransaction(env, "joe", balanceCol);
-    tx2.get().row("joe").col(balanceCol);
-    tx2.mutate().row("joe").col(balanceCol).set(21);
-    tx2.mutate().row("bob").col(balanceCol).set(11);
+    TestTransaction tx2 = new TestTransaction(env, "joe", BALANCE);
+    tx2.gets("joe", BALANCE);
+    tx2.set("joe", BALANCE, "21");
+    tx2.set("bob", BALANCE, "11");
 
-    TestTransaction tx1 = new TestTransaction(env, "joe", balanceCol);
-    tx1.get().row("joe").col(balanceCol);
-    tx1.mutate().row("jill").col(balanceCol).set(61);
+    TestTransaction tx1 = new TestTransaction(env, "joe", BALANCE);
+    tx1.gets("joe", BALANCE);
+    tx1.set("jill", BALANCE, "61");
 
     tx1.done();
     assertAAck(tx2);
 
     TestTransaction tx3 = new TestTransaction(env);
 
-    Assert.assertEquals(10, tx3.get().row("bob").col(balanceCol).toInteger(0));
-    Assert.assertEquals(20, tx3.get().row("joe").col(balanceCol).toInteger(0));
-    Assert.assertEquals(61, tx3.get().row("jill").col(balanceCol).toInteger(0));
+    Assert.assertEquals("10", tx3.gets("bob", BALANCE));
+    Assert.assertEquals("20", tx3.gets("joe", BALANCE));
+    Assert.assertEquals("61", tx3.gets("jill", BALANCE));
 
     tx3.done();
   }
@@ -426,13 +403,13 @@ public class FluoIT extends ITBaseImpl {
 
     env.setAuthorizations(new Authorizations("A", "B", "C"));
 
-    Column balanceCol = typeLayer.bc().fam("account").qual("balance").vis("A|B");
+    Column balanceCol = new Column("account", "balance", "A|B");
 
     TestTransaction tx = new TestTransaction(env);
 
-    tx.mutate().row("bob").col(balanceCol).set(10);
-    tx.mutate().row("joe").col(balanceCol).set(20);
-    tx.mutate().row("jill").col(balanceCol).set(60);
+    tx.set("bob", balanceCol, "10");
+    tx.set("joe", balanceCol, "20");
+    tx.set("jill", balanceCol, "60");
 
     tx.done();
 
@@ -441,9 +418,9 @@ public class FluoIT extends ITBaseImpl {
     env2.setAuthorizations(new Authorizations("B"));
 
     TestTransaction tx2 = new TestTransaction(env2);
-    Assert.assertEquals(10, tx2.get().row("bob").col(balanceCol).toInteger(0));
-    Assert.assertEquals(20, tx2.get().row("joe").col(balanceCol).toInteger(0));
-    Assert.assertEquals(60, tx2.get().row("jill").col(balanceCol).toInteger(0));
+    Assert.assertEquals("10", tx2.gets("bob", balanceCol));
+    Assert.assertEquals("20", tx2.gets("joe", balanceCol));
+    Assert.assertEquals("60", tx2.gets("jill", balanceCol));
     tx2.done();
     env2.close();
 
@@ -451,9 +428,9 @@ public class FluoIT extends ITBaseImpl {
     env3.setAuthorizations(new Authorizations("C"));
 
     TestTransaction tx3 = new TestTransaction(env3);
-    Assert.assertNull(tx3.get().row("bob").col(balanceCol).toInteger());
-    Assert.assertNull(tx3.get().row("joe").col(balanceCol).toInteger());
-    Assert.assertNull(tx3.get().row("jill").col(balanceCol).toInteger());
+    Assert.assertNull(tx3.gets("bob", balanceCol));
+    Assert.assertNull(tx3.gets("joe", balanceCol));
+    Assert.assertNull(tx3.gets("jill", balanceCol));
     tx3.done();
     env3.close();
   }
@@ -464,17 +441,17 @@ public class FluoIT extends ITBaseImpl {
     // status
 
     TestTransaction tx = new TestTransaction(env);
-    tx.mutate().row("d00001").fam("data").qual("content")
-        .set("blah blah, blah http://a.com. Blah blah http://b.com.  Blah http://c.com");
-    tx.mutate().row("d00001").fam("outlink").qual("http://a.com").set("");
-    tx.mutate().row("d00001").fam("outlink").qual("http://b.com").set("");
-    tx.mutate().row("d00001").fam("outlink").qual("http://c.com").set("");
-
-    tx.mutate().row("d00002").fam("data").qual("content")
-        .set("blah blah, blah http://d.com. Blah blah http://e.com.  Blah http://c.com");
-    tx.mutate().row("d00002").fam("outlink").qual("http://d.com").set("");
-    tx.mutate().row("d00002").fam("outlink").qual("http://e.com").set("");
-    tx.mutate().row("d00002").fam("outlink").qual("http://c.com").set("");
+    tx.set("d00001", new Column("data", "content"),
+        "blah blah, blah http://a.com. Blah blah http://b.com.  Blah http://c.com");
+    tx.set("d00001", new Column("outlink", "http://a.com"), "");
+    tx.set("d00001", new Column("outlink", "http://b.com"), "");
+    tx.set("d00001", new Column("outlink", "http://c.com"), "");
+
+    tx.set("d00002", new Column("data", "content"),
+        "blah blah, blah http://d.com. Blah blah http://e.com.  Blah http://c.com");
+    tx.set("d00002", new Column("outlink", "http://d.com"), "");
+    tx.set("d00002", new Column("outlink", "http://e.com"), "");
+    tx.set("d00002", new Column("outlink", "http://c.com"), "");
 
     tx.done();
 
@@ -482,12 +459,12 @@ public class FluoIT extends ITBaseImpl {
 
     TestTransaction tx3 = new TestTransaction(env);
 
-    tx3.mutate().row("d00001").fam("data").qual("content")
-        .set("blah blah, blah http://a.com. Blah http://c.com .  Blah http://z.com");
-    tx3.mutate().row("d00001").fam("outlink").qual("http://a.com").set("");
-    tx3.mutate().row("d00001").fam("outlink").qual("http://b.com").delete();
-    tx3.mutate().row("d00001").fam("outlink").qual("http://c.com").set("");
-    tx3.mutate().row("d00001").fam("outlink").qual("http://z.com").set("");
+    tx3.set("d00001", new Column("data", "content"),
+        "blah blah, blah http://a.com. Blah http://c.com .  Blah http://z.com");
+    tx3.set("d00001", new Column("outlink", "http://a.com"), "");
+    tx3.delete("d00001", new Column("outlink", "http://b.com"));
+    tx3.set("d00001", new Column("outlink", "http://c.com"), "");
+    tx3.set("d00001", new Column("outlink", "http://z.com"), "");
 
     tx3.done();
 
@@ -505,9 +482,9 @@ public class FluoIT extends ITBaseImpl {
     tx2.done();
 
     HashSet<Column> expected = new HashSet<>();
-    expected.add(typeLayer.bc().fam("outlink").qual("http://a.com").vis());
-    expected.add(typeLayer.bc().fam("outlink").qual("http://b.com").vis());
-    expected.add(typeLayer.bc().fam("outlink").qual("http://c.com").vis());
+    expected.add(new Column("outlink", "http://a.com"));
+    expected.add(new Column("outlink", "http://b.com"));
+    expected.add(new Column("outlink", "http://c.com"));
 
     Assert.assertEquals(expected, columns);
 
@@ -522,8 +499,8 @@ public class FluoIT extends ITBaseImpl {
         columns.add(citer.next().getKey());
       }
     }
-    expected.add(typeLayer.bc().fam("outlink").qual("http://z.com").vis());
-    expected.remove(typeLayer.bc().fam("outlink").qual("http://b.com").vis());
+    expected.add(new Column("outlink", "http://z.com"));
+    expected.remove(new Column("outlink", "http://b.com"));
     Assert.assertEquals(expected, columns);
     tx4.done();
   }

http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/b2c91b95/modules/integration/src/test/java/org/apache/fluo/integration/impl/GarbageCollectionIteratorIT.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/impl/GarbageCollectionIteratorIT.java b/modules/integration/src/test/java/org/apache/fluo/integration/impl/GarbageCollectionIteratorIT.java
index d83c6f1..0451f16 100644
--- a/modules/integration/src/test/java/org/apache/fluo/integration/impl/GarbageCollectionIteratorIT.java
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/impl/GarbageCollectionIteratorIT.java
@@ -90,14 +90,17 @@ public class GarbageCollectionIteratorIT extends ITBaseImpl {
 
   @Test(timeout = 60000)
   public void testDeletedDataIsDropped() throws Exception {
+
+    final Column docUri = new Column("doc", "uri");
+
     TestTransaction tx1 = new TestTransaction(env);
-    tx1.mutate().row("001").fam("doc").qual("uri").set("file:///abc.txt");
+    tx1.set("001", docUri, "file:///abc.txt");
     tx1.done();
 
     TestTransaction tx2 = new TestTransaction(env);
 
     TestTransaction tx3 = new TestTransaction(env);
-    tx3.mutate().row("001").fam("doc").qual("uri").delete();
+    tx3.delete("001", docUri);
     tx3.done();
 
     TestTransaction tx4 = new TestTransaction(env);
@@ -107,16 +110,16 @@ public class GarbageCollectionIteratorIT extends ITBaseImpl {
     // Force a garbage collection
     conn.tableOperations().compact(table, null, null, true, true);
 
-    Assert.assertEquals("file:///abc.txt", tx2.get().row("001").fam("doc").qual("uri").toString());
+    Assert.assertEquals("file:///abc.txt", tx2.gets("001", docUri));
 
     tx2.done();
 
-    Assert.assertNull(tx4.get().row("001").fam("doc").qual("uri").toString());
+    Assert.assertNull(tx4.gets("001", docUri));
 
     waitForGcTime(tx4.getStartTimestamp());
     conn.tableOperations().compact(table, null, null, true, true);
 
-    Assert.assertNull(tx4.get().row("001").fam("doc").qual("uri").toString());
+    Assert.assertNull(tx4.gets("001", docUri));
 
     Scanner scanner = conn.createScanner(table, Authorizations.EMPTY);
     Assert.assertEquals(0, Iterables.size(scanner));
@@ -134,8 +137,8 @@ public class GarbageCollectionIteratorIT extends ITBaseImpl {
     TestTransaction tx2 = new TestTransaction(env, t2);
 
     for (int r = 0; r < 10; r++) {
-      tx2.mutate().row(r + "").col(col1).set("1" + r + "0");
-      tx2.mutate().row(r + "").col(col2).set("1" + r + "1");
+      tx2.set(r + "", col1, "1" + r + "0");
+      tx2.set(r + "", col2, "1" + r + "1");
     }
 
     CommitData cd = tx2.createCommitData();

http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/b2c91b95/modules/integration/src/test/java/org/apache/fluo/integration/impl/NotificationGcIT.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/impl/NotificationGcIT.java b/modules/integration/src/test/java/org/apache/fluo/integration/impl/NotificationGcIT.java
index 81a5a93..37bca0a 100644
--- a/modules/integration/src/test/java/org/apache/fluo/integration/impl/NotificationGcIT.java
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/impl/NotificationGcIT.java
@@ -25,6 +25,7 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.fluo.accumulo.util.ColumnConstants;
 import org.apache.fluo.api.config.ObserverConfiguration;
+import org.apache.fluo.api.data.Column;
 import org.apache.fluo.core.impl.Environment;
 import org.apache.fluo.core.impl.Notification;
 import org.apache.fluo.core.util.ByteUtil;
@@ -36,7 +37,7 @@ import org.junit.Test;
 
 public class NotificationGcIT extends ITBaseMini {
 
-  public static void assertRawNotifications(int expected, Environment env) throws Exception {
+  private static void assertRawNotifications(int expected, Environment env) throws Exception {
     Scanner scanner = env.getConnector().createScanner(env.getTable(), env.getAuthorizations());
     scanner.fetchColumnFamily(ByteUtil.toText(ColumnConstants.NOTIFY_CF));
     int size = Iterables.size(scanner);
@@ -48,7 +49,7 @@ public class NotificationGcIT extends ITBaseMini {
     Assert.assertEquals(expected, size);
   }
 
-  public static int countNotifications(Environment env) throws Exception {
+  private static int countNotifications(Environment env) throws Exception {
     Scanner scanner = env.getConnector().createScanner(env.getTable(), env.getAuthorizations());
     Notification.configureScanner(scanner);
     return Iterables.size(scanner);
@@ -61,31 +62,35 @@ public class NotificationGcIT extends ITBaseMini {
 
   @Test
   public void testNotificationGC() throws Exception {
+
+    final Column statCount = new Column("stat", "count");
+    final Column statCheck = new Column("stat", "check");
+
     Environment env = new Environment(config);
 
     TestTransaction tx1 = new TestTransaction(env);
-    tx1.mutate().row("r1").fam("stat").qual("count").set(3);
+    tx1.set("r1", statCount, 3 + "");
     tx1.done();
 
     TestTransaction tx2 = new TestTransaction(env);
-    tx2.mutate().row("r2").fam("stat").qual("count").set(7);
+    tx2.set("r2", statCount, 7 + "");
     tx2.done();
 
     TestTransaction tx3 = new TestTransaction(env);
-    tx3.mutate().row("r1").fam("stats").qual("af89").set(5);
-    tx3.mutate().row("r1").fam("stat").qual("check").weaklyNotify();
+    tx3.set("r1", new Column("stats", "af89"), 5 + "");
+    tx3.setWeakNotification("r1", statCheck);
     tx3.done();
 
     TestTransaction tx4 = new TestTransaction(env);
-    tx4.mutate().row("r2").fam("stats").qual("af99").set(7);
-    tx4.mutate().row("r2").fam("stat").qual("check").weaklyNotify();
+    tx4.set("r2", new Column("stats", "af99"), 7 + "");
+    tx4.setWeakNotification("r2", statCheck);
     tx4.done();
 
     miniFluo.waitForObservers();
 
     TestTransaction tx5 = new TestTransaction(env);
-    Assert.assertEquals(8, tx5.get().row("r1").fam("stat").qual("count").toInteger(0));
-    Assert.assertEquals(14, tx5.get().row("r2").fam("stat").qual("count").toInteger(0));
+    Assert.assertEquals("8", tx5.gets("r1", statCount));
+    Assert.assertEquals("14", tx5.gets("r2", statCount));
 
     assertRawNotifications(4, env);
     Assert.assertEquals(0, countNotifications(env));

http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/b2c91b95/modules/integration/src/test/java/org/apache/fluo/integration/impl/ObserverConfigIT.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/impl/ObserverConfigIT.java b/modules/integration/src/test/java/org/apache/fluo/integration/impl/ObserverConfigIT.java
index 79b70e8..9766f9b 100644
--- a/modules/integration/src/test/java/org/apache/fluo/integration/impl/ObserverConfigIT.java
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/impl/ObserverConfigIT.java
@@ -20,24 +20,20 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.fluo.api.client.Snapshot;
+import org.apache.fluo.api.client.Transaction;
 import org.apache.fluo.api.client.TransactionBase;
 import org.apache.fluo.api.config.ObserverConfiguration;
 import org.apache.fluo.api.data.Bytes;
 import org.apache.fluo.api.data.Column;
 import org.apache.fluo.api.observer.AbstractObserver;
 import org.apache.fluo.api.observer.Observer.NotificationType;
-import org.apache.fluo.api.types.StringEncoder;
-import org.apache.fluo.api.types.TypeLayer;
-import org.apache.fluo.api.types.TypedSnapshot;
-import org.apache.fluo.api.types.TypedTransaction;
 import org.apache.fluo.integration.ITBaseMini;
 import org.junit.Assert;
 import org.junit.Test;
 
 public class ObserverConfigIT extends ITBaseMini {
 
-  private static TypeLayer tl = new TypeLayer(new StringEncoder());
-
   public static class ConfigurableObserver extends AbstractObserver {
 
     private ObservedColumn observedColumn;
@@ -48,7 +44,7 @@ public class ObserverConfigIT extends ITBaseMini {
     public void init(Context context) {
       String ocTokens[] = context.getParameters().get("observedCol").split(":");
       observedColumn =
-          new ObservedColumn(tl.bc().fam(ocTokens[0]).qual(ocTokens[1]).vis(),
+          new ObservedColumn(new Column(ocTokens[0], ocTokens[1]),
               NotificationType.valueOf(ocTokens[2]));
       outputCQ = Bytes.of(context.getParameters().get("outputCQ"));
       String swn = context.getParameters().get("setWeakNotification");
@@ -78,7 +74,7 @@ public class ObserverConfigIT extends ITBaseMini {
     }
   }
 
-  Map<String, String> newMap(String... args) {
+  private Map<String, String> newMap(String... args) {
     HashMap<String, String> ret = new HashMap<>();
     for (int i = 0; i < args.length; i += 2) {
       ret.put(args[i], args[i + 1]);
@@ -107,18 +103,18 @@ public class ObserverConfigIT extends ITBaseMini {
 
   @Test
   public void testObserverConfig() throws Exception {
-    try (TypedTransaction tx1 = tl.wrap(client.newTransaction())) {
-      tx1.mutate().row("r1").fam("fam1").qual("col1").set("abcdefg");
+    try (Transaction tx1 = client.newTransaction()) {
+      tx1.set("r1", new Column("fam1", "col1"), "abcdefg");
       tx1.commit();
     }
 
     miniFluo.waitForObservers();
 
-    try (TypedSnapshot tx2 = tl.wrap(client.newSnapshot())) {
-      Assert.assertNull(tx2.get().row("r1").fam("fam1").qual("col1").toString());
-      Assert.assertNull(tx2.get().row("r1").fam("fam1").qual("col2").toString());
-      Assert.assertNull(tx2.get().row("r1").fam("fam1").qual("col3").toString());
-      Assert.assertEquals("abcdefg", tx2.get().row("r1").fam("fam1").qual("col4").toString());
+    try (Snapshot tx2 = client.newSnapshot()) {
+      Assert.assertNull(tx2.gets("r1", new Column("fam1", "col1")));
+      Assert.assertNull(tx2.gets("r1", new Column("fam1", "col2")));
+      Assert.assertNull(tx2.gets("r1", new Column("fam1", "col3")));
+      Assert.assertEquals("abcdefg", tx2.gets("r1", new Column("fam1", "col4")));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/b2c91b95/modules/integration/src/test/java/org/apache/fluo/integration/impl/OracleIT.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/impl/OracleIT.java b/modules/integration/src/test/java/org/apache/fluo/integration/impl/OracleIT.java
index 857b48d..91636f7 100644
--- a/modules/integration/src/test/java/org/apache/fluo/integration/impl/OracleIT.java
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/impl/OracleIT.java
@@ -42,9 +42,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
-/**
- *
- */
 public class OracleIT extends ITBaseImpl {
 
   @Test