You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gora.apache.org by le...@apache.org on 2014/08/15 20:48:23 UTC

git commit: GORA-353 Accumulo authentication token serialized incorrectly

Repository: gora
Updated Branches:
  refs/heads/master 3cacf2a25 -> 92b884158


GORA-353 Accumulo authentication token serialized incorrectly


Project: http://git-wip-us.apache.org/repos/asf/gora/repo
Commit: http://git-wip-us.apache.org/repos/asf/gora/commit/92b88415
Tree: http://git-wip-us.apache.org/repos/asf/gora/tree/92b88415
Diff: http://git-wip-us.apache.org/repos/asf/gora/diff/92b88415

Branch: refs/heads/master
Commit: 92b88415820001da61e0a94aa526db3622b576dc
Parents: 3cacf2a
Author: Lewis John McGibbney <le...@jpl.nasa.gov>
Authored: Fri Aug 15 11:46:43 2014 -0700
Committer: Lewis John McGibbney <le...@jpl.nasa.gov>
Committed: Fri Aug 15 11:46:43 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   2 +
 gora-accumulo/pom.xml                           | 304 ++++++++++---------
 .../gora/accumulo/store/AccumuloStore.java      |  42 ++-
 .../accumulo/store/AuthenticationTokenTest.java |  90 ++++++
 4 files changed, 275 insertions(+), 163 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/gora/blob/92b88415/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c1fca0c..808b674 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -4,6 +4,8 @@
 
 Current Development 0.5-SNAPSHOT
 
+* GORA-353 Accumulo authentication token serialized incorrectly (Chin Huang via lewismc)
+
 * GORA-167 forward port of Make Cassandra keyspace consistency configurable within gora.properties (rmarroquin via lewismc)
 
 * GORA-364 MemStore.get fails with NPE when key is not set (Gerhard Gossen via lewismc)

http://git-wip-us.apache.org/repos/asf/gora/blob/92b88415/gora-accumulo/pom.xml
----------------------------------------------------------------------
diff --git a/gora-accumulo/pom.xml b/gora-accumulo/pom.xml
index ff0956b..b588a7d 100644
--- a/gora-accumulo/pom.xml
+++ b/gora-accumulo/pom.xml
@@ -1,145 +1,169 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<!-- 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. -->
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.gora</groupId>
-		<artifactId>gora</artifactId>
-		<version>0.5-SNAPSHOT</version>
-		<relativePath>../</relativePath>
-	</parent>
-	<artifactId>gora-accumulo</artifactId>
-	<packaging>bundle</packaging>
-
-	<name>Apache Gora :: Accumulo</name>
-        <url>http://gora.apache.org</url>
-    <description>The Apache Gora open source framework provides an in-memory data model and 
-    persistence for big data. Gora supports persisting to column stores, key value stores, 
-    document stores and RDBMSs, and analyzing the data with extensive Apache Hadoop MapReduce 
+  <!--
+  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.
+  -->
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.gora</groupId>
+    <artifactId>gora</artifactId>
+    <version>0.5-SNAPSHOT</version>
+    <relativePath>../</relativePath>
+  </parent>
+  <artifactId>gora-accumulo</artifactId>
+  <packaging>bundle</packaging>
+
+  <name>Apache Gora :: Accumulo</name>
+  <url>http://gora.apache.org</url>
+  <description>The Apache Gora open source framework provides an in-memory data model and
+    persistence for big data. Gora supports persisting to column stores, key value stores,
+    document stores and RDBMSs, and analyzing the data with extensive Apache Hadoop MapReduce
     support.</description>
-    <inceptionYear>2010</inceptionYear>
-    <organization>
-    	<name>The Apache Software Foundation</name>
-    	<url>http://www.apache.org/</url>
-    </organization>
-    <issueManagement>
-    	<system>JIRA</system>
-    	<url>https://issues.apache.org/jira/browse/GORA</url>
-    </issueManagement>
-    <ciManagement>
-    	<system>Jenkins</system>
-    	<url>https://builds.apache.org/job/Gora-trunk/</url>
-    </ciManagement>
-	
-    <properties>
-        <osgi.import>*</osgi.import>
-        <osgi.export>org.apache.gora.accumulo*;version="${project.version}";-noimport:=true</osgi.export>
-    </properties>
-    
-    <build>
-        <directory>target</directory>
-        <outputDirectory>target/classes</outputDirectory>
-        <finalName>${project.artifactId}-${project.version}</finalName>
-        <testOutputDirectory>target/test-classes</testOutputDirectory>
-        <testSourceDirectory>src/test/java</testSourceDirectory>
-        <sourceDirectory>src/main/java</sourceDirectory>
-        <testResources>
-          <testResource>
-            <directory>${project.basedir}/src/test/resources</directory>
-            <includes>
-              <include>**/*</include>
-            </includes>
-            <!--targetPath>${project.basedir}/target/classes/</targetPath-->
-          </testResource>
-        </testResources>
-        <plugins>
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>build-helper-maven-plugin</artifactId>
-                <version>${build-helper-maven-plugin.version}</version>
-                <executions>
-                    <execution>
-                        <phase>generate-sources</phase>
-                        <goals>
-                            <goal>add-source</goal>
-                        </goals>
-                        <configuration>
-                            <sources>
-                                <source>src/examples/java</source>
-                            </sources>
-                        </configuration>
-                    </execution>
-                </executions>
-            </plugin>
-        </plugins>
-    </build>
-
-    <dependencies>
-        <!-- Gora Internal Dependencies -->
-        <dependency>
-            <groupId>org.apache.gora</groupId>
-            <artifactId>gora-core</artifactId>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.gora</groupId>
-            <artifactId>gora-core</artifactId>
-            <type>test-jar</type>
-            <scope>test</scope>
-        </dependency>
-
-        <!--Accumulo Dependency -->
-        <dependency>
-           <groupId>org.apache.accumulo</groupId>
-           <artifactId>accumulo-core</artifactId>
-           <version>1.5.1</version>
-        </dependency>
-
-        <!-- Hadoop Dependencies -->
-        <dependency>
-            <groupId>org.apache.avro</groupId>
-            <artifactId>avro</artifactId>
-        </dependency>
-
-        <dependency>
-            <groupId>log4j</groupId>
-            <artifactId>log4j</artifactId>
-	        <exclusions>
-	          <exclusion>
-                <groupId>javax.jms</groupId>
-	            <artifactId>jms</artifactId>
-	          </exclusion>
-            </exclusions>
-        </dependency>
-
-        <!-- Testing Dependencies -->
-        <dependency>
-            <groupId>junit</groupId>
-            <artifactId>junit</artifactId>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.hadoop</groupId>
-            <artifactId>hadoop-test</artifactId>
-        </dependency>
-
-        <dependency>
-            <groupId>org.slf4j</groupId>
-            <artifactId>slf4j-simple</artifactId>
-            <scope>test</scope>
-        </dependency>
-           
-    </dependencies>
+  <inceptionYear>2010</inceptionYear>
+  <organization>
+    <name>The Apache Software Foundation</name>
+    <url>http://www.apache.org/</url>
+  </organization>
+  <issueManagement>
+    <system>JIRA</system>
+    <url>https://issues.apache.org/jira/browse/GORA</url>
+  </issueManagement>
+  <ciManagement>
+    <system>Jenkins</system>
+    <url>https://builds.apache.org/job/Gora-trunk/</url>
+  </ciManagement>
+
+  <properties>
+    <accumulo.version>1.5.1</accumulo.version>
+    <osgi.import>*</osgi.import>
+    <osgi.export>org.apache.gora.accumulo*;version="${project.version}";-noimport:=true</osgi.export>
+  </properties>
+
+  <build>
+    <directory>target</directory>
+    <outputDirectory>target/classes</outputDirectory>
+    <finalName>${project.artifactId}-${project.version}</finalName>
+    <testOutputDirectory>target/test-classes</testOutputDirectory>
+    <testSourceDirectory>src/test/java</testSourceDirectory>
+    <sourceDirectory>src/main/java</sourceDirectory>
+    <testResources>
+      <testResource>
+        <directory>${project.basedir}/src/test/resources</directory>
+        <includes>
+          <include>**/*</include>
+        </includes>
+        <!--targetPath>${project.basedir}/target/classes/</targetPath-->
+      </testResource>
+    </testResources>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>build-helper-maven-plugin</artifactId>
+        <version>${build-helper-maven-plugin.version}</version>
+        <executions>
+          <execution>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>add-source</goal>
+            </goals>
+            <configuration>
+              <sources>
+                <source>src/examples/java</source>
+              </sources>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+  <dependencies>
+    <!-- Gora Internal Dependencies -->
+    <dependency>
+      <groupId>org.apache.gora</groupId>
+      <artifactId>gora-core</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.gora</groupId>
+      <artifactId>gora-core</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+    <!--Accumulo Dependency -->
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-core</artifactId>
+      <version>${accumulo.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-minicluster</artifactId>
+      <version>${accumulo.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-io</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.zookeeper</groupId>
+      <artifactId>zookeeper</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- Hadoop Dependencies -->
+    <dependency>
+      <groupId>org.apache.avro</groupId>
+      <artifactId>avro</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>javax.jms</groupId>
+          <artifactId>jms</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!-- Testing Dependencies -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-test</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-simple</artifactId>
+      <scope>test</scope>
+    </dependency>
+
+  </dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/gora/blob/92b88415/gora-accumulo/src/main/java/org/apache/gora/accumulo/store/AccumuloStore.java
----------------------------------------------------------------------
diff --git a/gora-accumulo/src/main/java/org/apache/gora/accumulo/store/AccumuloStore.java b/gora-accumulo/src/main/java/org/apache/gora/accumulo/store/AccumuloStore.java
index 3c1911a..fff12be 100644
--- a/gora-accumulo/src/main/java/org/apache/gora/accumulo/store/AccumuloStore.java
+++ b/gora-accumulo/src/main/java/org/apache/gora/accumulo/store/AccumuloStore.java
@@ -68,6 +68,7 @@ import org.apache.accumulo.core.iterators.SortedKeyIterator;
 import org.apache.accumulo.core.iterators.user.TimestampFilter;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.security.CredentialHelper;
 import org.apache.accumulo.core.security.thrift.TCredentials;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.TextUtil;
@@ -106,7 +107,7 @@ import org.w3c.dom.Element;
 import org.w3c.dom.NodeList;
 
 /**
- * 
+ * Directs CRUD operations into Accumulo.
  */
 public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T> {
 
@@ -250,7 +251,7 @@ public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T
         e.printStackTrace();
         return toBytes(o);
       }
-    } else {     
+    } else {
       return toBytes(o);
     }
   }
@@ -357,20 +358,15 @@ public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T
       }
 
       try {
-        AuthenticationToken token =  new PasswordToken(password);
+        AuthenticationToken token = new PasswordToken(password);
         if (mock == null || !mock.equals("true")) {
           String instance = DataStoreFactory.findProperty(properties, this, INSTANCE_NAME_PROPERTY, null);
           String zookeepers = DataStoreFactory.findProperty(properties, this, ZOOKEEPERS_NAME_PROPERTY, null);
-          credentials = new TCredentials(user, 
-              "org.apache.accumulo.core.client.security.tokens.PasswordToken", 
-              ByteBuffer.wrap(password.getBytes()), instance);
           conn = new ZooKeeperInstance(instance, zookeepers).getConnector(user, token);
         } else {
-          conn = new MockInstance().getConnector(user, new PasswordToken(password));
-          credentials = new TCredentials(user, 
-              "org.apache.accumulo.core.client.security.tokens.PasswordToken", 
-              ByteBuffer.wrap(password.getBytes()), conn.getInstance().getInstanceID());
+          conn = new MockInstance().getConnector(user, token);
         }
+        credentials = CredentialHelper.create(user, token, conn.getInstance().getInstanceID());
 
         if (autoCreateSchema)
           createSchema();
@@ -379,7 +375,7 @@ public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T
       } catch (AccumuloSecurityException e) {
         throw new IOException(e);
       }
-    }catch(IOException e){
+    } catch(IOException e){
       LOG.error(e.getMessage(), e);
     }
   }
@@ -483,7 +479,7 @@ public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T
       LOG.error(e.getMessage(), e);
     } catch (TableNotFoundException e) {
       LOG.error(e.getMessage(), e);
-    } 
+    }
   }
 
   @Override
@@ -515,7 +511,7 @@ public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T
 
       if (currentMap != null) {
         if (currentFam.equals(entry.getKey().getColumnFamily())) {
-          currentMap.put(new Utf8(entry.getKey().getColumnQualifierData().toArray()), 
+          currentMap.put(new Utf8(entry.getKey().getColumnQualifierData().toArray()),
               fromBytes(currentSchema, entry.getValue().get()));
           continue;
         } else {
@@ -539,7 +535,7 @@ public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T
         currentFam = entry.getKey().getColumnFamily();
         currentSchema = field.schema().getValueType();
 
-        currentMap.put(new Utf8(entry.getKey().getColumnQualifierData().toArray()), 
+        currentMap.put(new Utf8(entry.getKey().getColumnQualifierData().toArray()),
             fromBytes(currentSchema, entry.getValue().get()));
         break;
       case ARRAY:
@@ -572,7 +568,7 @@ public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T
           currentFam = entry.getKey().getColumnFamily();
           currentSchema = effectiveSchema.getValueType();
 
-          currentMap.put(new Utf8(entry.getKey().getColumnQualifierData().toArray()), 
+          currentMap.put(new Utf8(entry.getKey().getColumnQualifierData().toArray()),
               fromBytes(currentSchema, entry.getValue().get()));
           break;
         }
@@ -603,7 +599,7 @@ public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T
    * @return String The field name
    */
   private String getFieldName(Entry<Key, Value> entry) {
-    String fieldName = mapping.columnMap.get(new Pair<Text,Text>(entry.getKey().getColumnFamily(), 
+    String fieldName = mapping.columnMap.get(new Pair<Text,Text>(entry.getKey().getColumnFamily(),
         entry.getKey().getColumnQualifier()));
     if (fieldName == null) {
       fieldName = mapping.columnMap.get(new Pair<Text,Text>(entry.getKey().getColumnFamily(), null));
@@ -667,7 +663,7 @@ public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T
         }
         Field field = fields.get(i);
 
-        Object o = val.get(field.pos());       
+        Object o = val.get(field.pos());
 
         Pair<Text,Text> col = mapping.fieldMap.get(field.name());
 
@@ -735,11 +731,11 @@ public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T
     if (o == null){
       return 0;
     }
-    
+
     Set<?> es = ((Map<?, ?>)o).entrySet();
     for (Object entry : es) {
       Object mapKey = ((Entry<?, ?>) entry).getKey();
-      Object mapVal = ((Entry<?, ?>) entry).getValue();                  
+      Object mapVal = ((Entry<?, ?>) entry).getValue();
       if ((o instanceof DirtyMapWrapper && ((DirtyMapWrapper<?, ?>)o).isDirty())
           || !(o instanceof DirtyMapWrapper)) { //mapVal instanceof Dirtyable && ((Dirtyable)mapVal).isDirty()) {
         m.put(col.getFirst(), new Text(toBytes(mapKey)), new Value(toBytes(valueType, mapVal)));
@@ -763,7 +759,7 @@ public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T
     if (o == null){
       return 0;
     }
-    
+
     List<?> array = (List<?>) o;  // both GenericArray and DirtyListWrapper
     int j = 0;
     for (Object item : array) {
@@ -862,7 +858,7 @@ public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T
       // TODO return empty result?
       LOG.error(e.getMessage(), e);
       return null;
-    } 
+    }
   }
 
   @Override
@@ -1027,7 +1023,7 @@ public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T
       }
     } catch (MutationsRejectedException e) {
       LOG.error(e.getMessage(), e);
-    } 
+    }
   }
 
   @Override
@@ -1039,6 +1035,6 @@ public class AccumuloStore<K,T extends PersistentBase> extends DataStoreBase<K,T
       }
     } catch (MutationsRejectedException e) {
       LOG.error(e.getMessage(), e);
-    } 
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/gora/blob/92b88415/gora-accumulo/src/test/java/org/apache/gora/accumulo/store/AuthenticationTokenTest.java
----------------------------------------------------------------------
diff --git a/gora-accumulo/src/test/java/org/apache/gora/accumulo/store/AuthenticationTokenTest.java b/gora-accumulo/src/test/java/org/apache/gora/accumulo/store/AuthenticationTokenTest.java
new file mode 100644
index 0000000..ecb8da9
--- /dev/null
+++ b/gora-accumulo/src/test/java/org/apache/gora/accumulo/store/AuthenticationTokenTest.java
@@ -0,0 +1,90 @@
+/**
+ * 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.gora.accumulo.store;
+
+import java.util.List;
+import java.util.Properties;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
+import org.apache.gora.examples.generated.Employee;
+import org.apache.gora.query.PartitionQuery;
+import org.apache.gora.store.DataStore;
+import org.apache.gora.store.DataStoreFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests authentication token is serialized correctly.
+ */
+public class AuthenticationTokenTest {
+  private static final Logger LOG = LoggerFactory.getLogger(AuthenticationTokenTest.class);
+
+  private static final String GORA_DATASTORE =
+      DataStoreFactory.GORA + "." + DataStoreFactory.DATASTORE + ".";
+  private static final String PASSWORD = "password";
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  private MiniAccumuloCluster cluster;
+  private DataStore<String, Employee> employeeStore;
+
+  @Before
+  @SuppressWarnings("unchecked")
+  public void setUp() throws Exception {
+    cluster = new MiniAccumuloCluster(temporaryFolder.getRoot(), PASSWORD);
+    cluster.start();
+
+    Properties properties = DataStoreFactory.createProps();
+    properties.setProperty(
+        GORA_DATASTORE + AccumuloStore.MOCK_PROPERTY,
+        "false");
+    properties.setProperty(
+        GORA_DATASTORE + AccumuloStore.INSTANCE_NAME_PROPERTY,
+        cluster.getInstanceName());
+    properties.setProperty(
+        GORA_DATASTORE + AccumuloStore.ZOOKEEPERS_NAME_PROPERTY,
+        cluster.getZooKeepers());
+    properties.setProperty(
+        GORA_DATASTORE + AccumuloStore.PASSWORD_PROPERTY,
+        PASSWORD);
+
+    employeeStore = DataStoreFactory.createDataStore(
+        AccumuloStore.class,
+        String.class,
+        Employee.class,
+        new Configuration(),
+        properties);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    cluster.stop();
+  }
+
+  @Test
+  public void testAuthenticationTokenIsSerializedCorrectly() throws Exception {
+    List<PartitionQuery<String, Employee>> partitions =
+        employeeStore.getPartitions(employeeStore.newQuery());
+    LOG.debug("partitions {}", partitions);
+  }
+}