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 2013/12/03 22:27:55 UTC

svn commit: r1547591 - in /gora/branches/GORA_94: ./ gora-accumulo/src/main/java/org/apache/gora/accumulo/store/ gora-cassandra/src/main/java/org/apache/gora/cassandra/store/ gora-core/src/examples/avro/ gora-core/src/examples/java/org/apache/gora/exam...

Author: lewismc
Date: Tue Dec  3 21:27:54 2013
New Revision: 1547591

URL: http://svn.apache.org/r1547591
Log:
GORA-246v4

Modified:
    gora/branches/GORA_94/CHANGES.txt
    gora/branches/GORA_94/gora-accumulo/src/main/java/org/apache/gora/accumulo/store/AccumuloStore.java
    gora/branches/GORA_94/gora-cassandra/src/main/java/org/apache/gora/cassandra/store/CassandraStore.java
    gora/branches/GORA_94/gora-core/src/examples/avro/webpage.json
    gora/branches/GORA_94/gora-core/src/examples/java/org/apache/gora/examples/WebPageDataCreator.java
    gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/memory/store/MemStore.java
    gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/persistency/impl/BeanFactoryImpl.java
    gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/query/impl/PartitionQueryImpl.java
    gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java
    gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/util/IOUtils.java
    gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/mapreduce/MapReduceTestUtils.java
    gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/store/DataStoreTestBase.java
    gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/store/DataStoreTestUtil.java
    gora/branches/GORA_94/gora-hbase/src/main/java/org/apache/gora/hbase/store/HBaseStore.java
    gora/branches/GORA_94/gora-hbase/src/test/java/org/apache/gora/hbase/store/TestHBaseStore.java
    gora/branches/GORA_94/gora-solr/src/main/java/org/apache/gora/solr/store/SolrStore.java

Modified: gora/branches/GORA_94/CHANGES.txt
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/CHANGES.txt?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/CHANGES.txt (original)
+++ gora/branches/GORA_94/CHANGES.txt Tue Dec  3 21:27:54 2013
@@ -6,6 +6,10 @@ Gora Change Log
 
 * GORA-94 Upgrade to Apache Avro 1.7.x  ==1st Attempt== (Ed Kohlwey via lewismc)
 
+* GORA-275 Update Gora stores to pass conf when creating instance of PartitionQueryImpl (Damien Raude-Morvan via hsaputra)
+
+* GORA-270 IOUtils static SerializationFactory field (Damien Raude-Morvan via hsaputra)
+
 * GORA-268 Make GoraCompiler the main manifest attribute in gora-core (Apostolos Giannakidis via lewismc)
 
 * GORA-265 Support for dynamic file extensions when traversing a directory (Apostolos Giannakidis via lewismc)

Modified: gora/branches/GORA_94/gora-accumulo/src/main/java/org/apache/gora/accumulo/store/AccumuloStore.java
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-accumulo/src/main/java/org/apache/gora/accumulo/store/AccumuloStore.java?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-accumulo/src/main/java/org/apache/gora/accumulo/store/AccumuloStore.java (original)
+++ gora/branches/GORA_94/gora-accumulo/src/main/java/org/apache/gora/accumulo/store/AccumuloStore.java Tue Dec  3 21:27:54 2013
@@ -778,6 +778,7 @@ public class AccumuloStore<K,T extends P
           }
           
           PartitionQueryImpl pqi = new PartitionQueryImpl<K,T>(query, startKey, endKey, new String[] {location});
+          pqi.setConf(getConf());
           ret.add(pqi);
         }
       }

Modified: gora/branches/GORA_94/gora-cassandra/src/main/java/org/apache/gora/cassandra/store/CassandraStore.java
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-cassandra/src/main/java/org/apache/gora/cassandra/store/CassandraStore.java?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-cassandra/src/main/java/org/apache/gora/cassandra/store/CassandraStore.java (original)
+++ gora/branches/GORA_94/gora-cassandra/src/main/java/org/apache/gora/cassandra/store/CassandraStore.java Tue Dec  3 21:27:54 2013
@@ -292,7 +292,9 @@ public class CassandraStore<K, T extends
       throws IOException {
     // just a single partition
     List<PartitionQuery<K,T>> partitions = new ArrayList<PartitionQuery<K,T>>();
-    partitions.add(new PartitionQueryImpl<K,T>(query));
+    PartitionQueryImpl<K, T> pqi = new PartitionQueryImpl<K, T>(query);
+    pqi.setConf(getConf());
+    partitions.add(pqi);
     return partitions;
   }
   

Modified: gora/branches/GORA_94/gora-core/src/examples/avro/webpage.json
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-core/src/examples/avro/webpage.json?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-core/src/examples/avro/webpage.json (original)
+++ gora/branches/GORA_94/gora-core/src/examples/avro/webpage.json Tue Dec  3 21:27:54 2013
@@ -12,9 +12,9 @@
       "type": "record",
       "namespace": "org.apache.gora.examples.generated",
       "fields" : [
-			  {"name":"version","type":"int","default":0},
-			  {"name":"data","type":{"type":"map","values":"string"}, "default":null}
-			]
+        {"name":"version","type":"int","default":0},
+        {"name":"data","type":{"type":"map","values":"string"}, "default":null}
+      ]
     }}
   ]
 }

Modified: gora/branches/GORA_94/gora-core/src/examples/java/org/apache/gora/examples/WebPageDataCreator.java
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-core/src/examples/java/org/apache/gora/examples/WebPageDataCreator.java?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-core/src/examples/java/org/apache/gora/examples/WebPageDataCreator.java (original)
+++ gora/branches/GORA_94/gora-core/src/examples/java/org/apache/gora/examples/WebPageDataCreator.java Tue Dec  3 21:27:54 2013
@@ -120,14 +120,13 @@ public class WebPageDataCreator {
         page = WebPage.newBuilder().build();
         page.setUrl(new Utf8(URLS[i]));
         page.setParsedContent(new ArrayList<CharSequence>());
-	    if (CONTENTS[i]!=null){
-		    page.setContent(ByteBuffer.wrap(CONTENTS[i].getBytes()));
-		    for(String token : CONTENTS[i].split(" ")) {
-		      page.getParsedContent().add(new Utf8(token));  
-		    }
-	    }
-        
         page.setOutlinks(new HashMap<CharSequence, CharSequence>());
+        if (CONTENTS[i]!=null){
+          page.setContent(ByteBuffer.wrap(CONTENTS[i].getBytes()));
+          for(String token : CONTENTS[i].split(" ")) {
+            page.getParsedContent().add(new Utf8(token));  
+          }
+        }
         for(int j=0; j<LINKS[i].length; j++) {
           page.getOutlinks().put(new Utf8(URLS[LINKS[i][j]]), new Utf8(ANCHORS[i][j]));
         }

Modified: gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/memory/store/MemStore.java
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/memory/store/MemStore.java?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/memory/store/MemStore.java (original)
+++ gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/memory/store/MemStore.java Tue Dec  3 21:27:54 2013
@@ -178,7 +178,9 @@ public class MemStore<K, T extends Persi
    */
   public List<PartitionQuery<K, T>> getPartitions(Query<K, T> query){
     List<PartitionQuery<K, T>> list = new ArrayList<PartitionQuery<K,T>>();
-    list.add(new PartitionQueryImpl<K, T>(query));
+    PartitionQueryImpl<K, T> pqi = new PartitionQueryImpl<K, T>(query);
+    pqi.setConf(getConf());
+    list.add(pqi);
     return list;
   }
 

Modified: gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/persistency/impl/BeanFactoryImpl.java
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/persistency/impl/BeanFactoryImpl.java?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/persistency/impl/BeanFactoryImpl.java (original)
+++ gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/persistency/impl/BeanFactoryImpl.java Tue Dec  3 21:27:54 2013
@@ -50,8 +50,8 @@ public class BeanFactoryImpl<K, T extend
   private boolean isKeyPersistent = false;
   
   /**
-   * 
-   * @param keyClass
+   * Default constructor for this class.
+   * @param keyClass.
    * @param persistentClass
    */
   public BeanFactoryImpl(Class<K> keyClass, Class<T> persistentClass) {

Modified: gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/query/impl/PartitionQueryImpl.java
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/query/impl/PartitionQueryImpl.java?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/query/impl/PartitionQueryImpl.java (original)
+++ gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/query/impl/PartitionQueryImpl.java Tue Dec  3 21:27:54 2013
@@ -131,7 +131,7 @@ public String[] getLocations() {
   @Override
   public void write(DataOutput out) throws IOException {
     super.write(out);
-    IOUtils.serialize(null, out, baseQuery);
+    IOUtils.serialize(getConf(), out, baseQuery);
     IOUtils.writeStringArray(out, locations);
   }
 
@@ -139,7 +139,7 @@ public String[] getLocations() {
   public void readFields(DataInput in) throws IOException {
     super.readFields(in);
     try {
-      baseQuery = IOUtils.deserialize(null, in, null);
+      baseQuery = IOUtils.deserialize(getConf(), in, null);
     } catch (ClassNotFoundException ex) {
       throw new IOException(ex);
     }

Modified: gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java (original)
+++ gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java Tue Dec  3 21:27:54 2013
@@ -222,9 +222,9 @@ public String[] getFields() {
     if(!nullFields[1])
       fields = IOUtils.readStringArray(in);
     if(!nullFields[2])
-      startKey = IOUtils.deserialize(null, in, null, dataStore.getKeyClass());
+      startKey = IOUtils.deserialize(getConf(), in, null, dataStore.getKeyClass());
     if(!nullFields[3])
-      endKey = IOUtils.deserialize(null, in, null, dataStore.getKeyClass());
+      endKey = IOUtils.deserialize(getConf(), in, null, dataStore.getKeyClass());
     if(!nullFields[4])
       filter = Text.readString(in);
 

Modified: gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/util/IOUtils.java
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/util/IOUtils.java?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/util/IOUtils.java (original)
+++ gora/branches/GORA_94/gora-core/src/main/java/org/apache/gora/util/IOUtils.java Tue Dec  3 21:27:54 2013
@@ -59,20 +59,12 @@ import org.apache.hadoop.io.serializer.S
  */
 public class IOUtils {
 
-  private static SerializationFactory serializationFactory = null;
-  private static Configuration conf;
-
   public static final int BUFFER_SIZE = 8192;
 
   private static BinaryDecoder decoder;
 
   private static Configuration getOrCreateConf(Configuration conf) {
-    if(conf == null) {
-      if(IOUtils.conf == null) {
-        IOUtils.conf = new Configuration();
-      }
-    }
-    return conf != null ? conf : IOUtils.conf;
+    return conf != null ? conf : new Configuration();
   }
 
   public static Object readObject(DataInput in)
@@ -111,9 +103,7 @@ public class IOUtils {
   public static<T> void serialize(Configuration conf, DataOutput out
       , T obj, Class<T> objClass) throws IOException {
 
-    if(serializationFactory == null) {
-      serializationFactory = new SerializationFactory(getOrCreateConf(conf));
-    }
+    SerializationFactory serializationFactory = new SerializationFactory(getOrCreateConf(conf));
     Serializer<T> serializer = serializationFactory.getSerializer(objClass);
 
     ByteBufferOutputStream os = new ByteBufferOutputStream();
@@ -200,9 +190,7 @@ public class IOUtils {
    * @throws IOException */
   public static<T> T deserialize(Configuration conf, DataInput in
       , T obj , Class<T> objClass) throws IOException {
-    if(serializationFactory == null) {
-      serializationFactory = new SerializationFactory(getOrCreateConf(conf));
-    }
+    SerializationFactory serializationFactory = new SerializationFactory(getOrCreateConf(conf));
     Deserializer<T> deserializer = serializationFactory.getDeserializer(
         objClass);
 

Modified: gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/mapreduce/MapReduceTestUtils.java
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/mapreduce/MapReduceTestUtils.java?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/mapreduce/MapReduceTestUtils.java (original)
+++ gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/mapreduce/MapReduceTestUtils.java Tue Dec  3 21:27:54 2013
@@ -42,7 +42,10 @@ public class MapReduceTestUtils {
 
   private static final Logger log = LoggerFactory.getLogger(MapReduceTestUtils.class);
   
-  /** Tests by running the {@link QueryCounter} mapreduce job */
+  /** 
+   * Tests by running the {@link org.apache.gora.examples.mapreduce.QueryCounter} 
+   * mapreduce job 
+   */
   public static void testCountQuery(DataStore<String, WebPage> dataStore, Configuration conf)
       throws Exception {
 
@@ -54,9 +57,9 @@ public class MapReduceTestUtils {
     QueryCounter<String,WebPage> counter = new QueryCounter<String,WebPage>(conf);
     Query<String,WebPage> query = dataStore.newQuery();
     List<Field> fields = WebPage.SCHEMA$.getFields();
-    String[] fieldNames = new String[fields.size()];
+    String[] fieldNames = new String[fields.size() - 1];
     for(int i = 0; i< fieldNames.length; i++){
-      fieldNames[i] = fields.get(i).name();
+      fieldNames[i] = fields.get(i+1).name();
     }
     query.setFields(fieldNames);
     

Modified: gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/store/DataStoreTestBase.java
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/store/DataStoreTestBase.java?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/store/DataStoreTestBase.java (original)
+++ gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/store/DataStoreTestBase.java Tue Dec  3 21:27:54 2013
@@ -237,7 +237,7 @@ public abstract class DataStoreTestBase 
 
   @Test
   /**
-   * Tests put and get a record with a double  nested recursive record
+   * Tests put and get a record with a double nested recursive record
    * Employee with a boss (nested).
    * @throws IOException
    * @throws Exception
@@ -249,8 +249,9 @@ public abstract class DataStoreTestBase 
 
   @Test
   /**
-   * Tests put and get a record with a nested record (not recursive)
-   * The webpage of an Employee
+   * Tests put and get of an {@link org.apache.gora.examples.generated.Employee} 
+   * record with a nested {@link org.apache.gora.examples.generated.WegPage} record (not recursive)
+   * the webpage of an Employee.
    * @throws IOException
    * @throws Exception
    */

Modified: gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/store/DataStoreTestUtil.java
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/store/DataStoreTestUtil.java?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/store/DataStoreTestUtil.java (original)
+++ gora/branches/GORA_94/gora-core/src/test/java/org/apache/gora/store/DataStoreTestUtil.java Tue Dec  3 21:27:54 2013
@@ -161,10 +161,9 @@ public class DataStoreTestUtil {
 
     Employee after = dataStore.get(ssn, AvroUtils.getSchemaFieldNames(Employee.SCHEMA$));
 
-    assertEquals(employee, after);
+    assertEqualEmployeeObjects(employee, after);
   }
 
-
   public static void testGetEmployeeRecursive(DataStore<String, Employee> dataStore)
     throws IOException, Exception {
 
@@ -175,9 +174,8 @@ public class DataStoreTestUtil {
     String ssn = employee.getSsn().toString();
     dataStore.put(ssn, employee);
     dataStore.flush();
-    Employee after = dataStore.get(ssn, getFields(Employee.SCHEMA$.getFields()));
-    assertEquals(employee, after);
-    assertEquals(boss, after.getBoss()) ;
+    Employee after = dataStore.get(ssn, AvroUtils.getSchemaFieldNames(Employee.SCHEMA$));
+    assertEqualEmployeeObjects(employee, after);
   }
 
   public static void testGetEmployeeDoubleRecursive(DataStore<String, Employee> dataStore)
@@ -193,10 +191,8 @@ public class DataStoreTestUtil {
       String ssn = employee.getSsn().toString();
       dataStore.put(ssn, employee);
       dataStore.flush();
-      Employee after = dataStore.get(ssn, getFields(Employee.SCHEMA$.getFields()));
-      assertEquals(employee, after);
-      assertEquals(boss, after.getBoss()) ;
-      assertEquals(uberBoss, ((Employee)after.getBoss()).getBoss()) ;
+      Employee after = dataStore.get(ssn, AvroUtils.getSchemaFieldNames(Employee.SCHEMA$));
+      assertEqualEmployeeObjects(employee, after);
     }
   
   public static void testGetEmployeeNested(DataStore<String, Employee> dataStore)
@@ -207,7 +203,10 @@ public class DataStoreTestUtil {
     
     webpage.setUrl(new Utf8("url..")) ;
     webpage.setContent(ByteBuffer.wrap("test content".getBytes())) ;
-    Metadata metadata = new BeanFactoryImpl<String,Metadata>(String.class,Metadata.class).newPersistent() ;
+    webpage.setParsedContent(new ArrayList<CharSequence>());
+    webpage.setOutlinks(new HashMap<CharSequence, CharSequence>());
+    Metadata metadata = new BeanFactoryImpl<String,Metadata>(String.class,Metadata.class).newPersistent();
+    metadata.setData(new HashMap<CharSequence, CharSequence>());
     webpage.setMetadata(metadata) ;
     employee.setWebpage(webpage) ;
     
@@ -215,9 +214,9 @@ public class DataStoreTestUtil {
    
     dataStore.put(ssn, employee);
     dataStore.flush();
-    Employee after = dataStore.get(ssn, getFields(Employee.SCHEMA$.getFields()));
-    assertEquals(employee, after);
-    assertEquals(webpage, after.getWebpage()) ;
+    Employee after = dataStore.get(ssn, AvroUtils.getSchemaFieldNames(Employee.SCHEMA$));
+    assertEqualEmployeeObjects(employee, after);
+    assertEqualWebPageObjects(webpage, after.getWebpage());
   }
   
   public static void testGetEmployee3UnionField(DataStore<String, Employee> dataStore)
@@ -229,8 +228,8 @@ public class DataStoreTestUtil {
     String ssn = employee.getSsn().toString();
     dataStore.put(ssn, employee);
     dataStore.flush();
-    Employee after = dataStore.get(ssn, getFields(Employee.SCHEMA$.getFields()));
-    assertEquals(employee, after);
+    Employee after = dataStore.get(ssn, AvroUtils.getSchemaFieldNames(Employee.SCHEMA$));
+    assertEqualEmployeeObjects(employee, after);
     assertEquals("Real boss", ((Utf8)after.getBoss()).toString()) ;
   }
   
@@ -258,9 +257,119 @@ public class DataStoreTestUtil {
         expected.put(index, employee.get(index));
       }
 
-      assertEquals(expected, after);
+      assertEqualEmployeeObjects(expected, after);
     }
   }
+  
+  /**
+   * Simple function which iterates through a before (put) and after (get) object
+   * in an attempt to verify if the same field's and values have been obtained.
+   * Within the original employee object we iterate from 1 instead of 0 due to the 
+   * removal of the '__g__' field at position 0 when we put objects into the datastore. 
+   * This field is used to identify whether fields within the object, and 
+   * consequently the object itself, are/is dirty however this field is not 
+   * required when persisting the object.
+   * We explicitly get values from each field as this makes it easier to debug 
+   * if tests go wrong.
+   * @param employee
+   * @param after
+   */
+  private static void assertEqualEmployeeObjects(Employee employee, Employee after) {
+    //for (int i = 1; i < employee.SCHEMA$.getFields().size(); i++) {
+    //  for (int j = 1; j < after.SCHEMA$.getFields().size(); j++) {
+    //    assertEquals(employee.SCHEMA$.getFields().get(i), after.SCHEMA$.getFields().get(j));
+    //  }
+    //}
+    //check name field
+    CharSequence beforeName = employee.getName();
+    CharSequence afterName = after.getName();
+    assertEquals(beforeName, afterName);
+    //check dateOfBirth field
+    Long beforeDOB = employee.getDateOfBirth();
+    Long afterDOB = after.getDateOfBirth();
+    assertEquals(beforeDOB, afterDOB);
+    //check ssn field
+    CharSequence beforeSsn = employee.getSsn();
+    CharSequence afterSsn = after.getSsn();
+    assertEquals(beforeSsn, afterSsn);
+    //check salary field
+    Integer beforeSalary = employee.getSalary();
+    Integer afterSalary = after.getSalary();
+    assertEquals(beforeSalary, afterSalary);
+    //check boss field
+    if (employee.getBoss() != null) {
+      if (employee.getBoss() instanceof Utf8) {
+        String beforeBoss = employee.getBoss().toString();
+        String afterBoss = after.getBoss().toString();
+        assertEquals("Boss String field values in UNION should be the same", 
+            beforeBoss, afterBoss);
+      } else {
+        Employee beforeBoss = (Employee) employee.getBoss();
+        Employee afterBoss = (Employee) after.getBoss();
+        assertEqualEmployeeObjects(beforeBoss, afterBoss);
+      }
+    }
+    //check webpage field
+    if (employee.getWebpage() != null) {
+      WebPage beforeWebPage = employee.getWebpage();
+      WebPage afterWebPage = after.getWebpage();
+      assertEqualWebPageObjects(beforeWebPage, afterWebPage);
+    }
+  }
+
+  /**
+   * Mimics {@link org.apache.gora.store.DataStoreTestUtil#assertEqualEmployeeObjects(Employee, Employee)}
+   * in that we pick our way through fields within before and after 
+   * {@link org.apache.gora.examples.generated.WebPage} objects comparing field values.
+   * @param beforeWebPage
+   * @param afterWebPage
+   */
+  private static void assertEqualWebPageObjects(WebPage beforeWebPage, WebPage afterWebPage) {
+    //check url field
+    CharSequence beforeUrl = beforeWebPage.getUrl();
+    CharSequence afterUrl = afterWebPage.getUrl();
+    assertEquals(beforeUrl, afterUrl);
+    //check content field
+    ByteBuffer beforeContent = beforeWebPage.getContent();
+    ByteBuffer afterContent = afterWebPage.getContent();
+    assertEquals(beforeContent, afterContent);
+    //check parsedContent field
+    List<CharSequence> beforeParsedContent = 
+        (List<CharSequence>) beforeWebPage.getParsedContent();
+    List<CharSequence> afterParsedContent = 
+        (List<CharSequence>) afterWebPage.getParsedContent();
+    assertEquals(beforeParsedContent, afterParsedContent);
+    //check outlinks field
+    Map<CharSequence, CharSequence> beforeOutlinks = 
+        (Map<java.lang.CharSequence,java.lang.CharSequence>) beforeWebPage.getOutlinks();
+    Map<CharSequence, CharSequence> afterOutlinks = 
+        (Map<java.lang.CharSequence,java.lang.CharSequence>) afterWebPage.getOutlinks();
+    assertEquals(beforeOutlinks, afterOutlinks);
+    //check metadata field
+    if (beforeWebPage.get(5) != null) {
+      Metadata beforeMetadata = beforeWebPage.getMetadata();
+      Metadata afterMetadata = afterWebPage.getMetadata();
+      assertEqualMetadataObjects(beforeMetadata, afterMetadata);
+    }
+  }
+
+  /**
+   * Mimics {@link org.apache.gora.store.DataStoreTestUtil#assertEqualEmployeeObjects(Employee, Employee)}
+   * in that we pick our way through fields within before and after 
+   * {@link org.apache.gora.examples.generated.Metadata} objects comparing field values.
+   * @param beforeMetadata
+   * @param afterMetadata
+   */
+  private static void assertEqualMetadataObjects(Metadata beforeMetadata, Metadata afterMetadata) {
+    //check version field
+    int beforeVersion = beforeMetadata.getVersion();
+    int afterVersion = afterMetadata.getVersion();
+    assertEquals(beforeVersion, afterVersion);
+    //check data field
+    Map<CharSequence, CharSequence> beforeData = beforeMetadata.getData();
+    Map<CharSequence, CharSequence> afterData =  afterMetadata.getData();
+    assertEquals(beforeData, afterData);
+  }
 
   public static Employee testPutEmployee(DataStore<String, Employee> dataStore)
   throws IOException, Exception {
@@ -299,6 +408,16 @@ public class DataStoreTestUtil {
     assertNull(employee);
   }
 
+  /**
+   * Here we create 5 {@link org.apache.gora.examples.generated.Employee} objects
+   * before populating fields with data and flushing them to the datastore.
+   * We then update the 1st of the {@link org.apache.gora.examples.generated.Employee}'s
+   * with more data and flush this data. Assertions are then made over the updated
+   * {@link org.apache.gora.examples.generated.Employee} object.
+   * @param dataStore
+   * @throws IOException
+   * @throws Exception
+   */
   public static void testUpdateEmployee(DataStore<String, Employee> dataStore)
   throws IOException, Exception {
     dataStore.createSchema();
@@ -319,7 +438,7 @@ public class DataStoreTestUtil {
     for (int i = 0; i < 1; i++) {
       Employee employee = Employee.newBuilder().build();
       employee.setName(new Utf8("John Doe " + (i + 5)));
-      employee.setDateOfBirth(now - 18L *  YEAR_IN_MS);
+      employee.setDateOfBirth(now - 18L * YEAR_IN_MS);
       employee.setSalary(120000);
       employee.setSsn(new Utf8(Long.toString(ssn + i)));
       dataStore.put(employee.getSsn().toString(), employee);
@@ -330,12 +449,25 @@ public class DataStoreTestUtil {
     for (int i = 0; i < 1; i++) {
       String key = Long.toString(ssn + i);
       Employee employee = dataStore.get(key);
-      assertEquals(now - 18L * YEAR_IN_MS, employee.getDateOfBirth().intValue()); 
+      assertEquals(now - 18L * YEAR_IN_MS, employee.getDateOfBirth().longValue()); 
       assertEquals("John Doe " + (i + 5), employee.getName().toString());
       assertEquals(120000, employee.getSalary().intValue()); 
     }
   }
 
+  /**
+   * Here we create 7 {@link org.apache.gora.examples.generated.WebPage}
+   * objects and populate field data before flushing the objects to the 
+   * datastore. We then get the objects, adding data to the 'content' and
+   * 'parsedContent' fields before clearing the 'outlinks' field and 
+   * re-populating it. This data is then flushed to the datastore. 
+   * Finally we get the {@link org.apache.gora.examples.generated.WebPage}
+   * objects and make various assertions over verious fields. This tests 
+   * that we can update fields and that data can be written and read correctly.
+   * @param dataStore
+   * @throws IOException
+   * @throws Exception
+   */
   public static void testUpdateWebPage(DataStore<String, WebPage> dataStore)
   throws IOException, Exception {
     dataStore.createSchema();
@@ -390,7 +522,7 @@ public class DataStoreTestUtil {
         j++;
       }
       int count = 0;
-      for (j = 1; j < urls.length; j += 2) {
+      for (j = 0; j < urls.length; j++) {
         CharSequence link = webPage.getOutlinks().get(new Utf8(anchor + j));
         assertNotNull(link);
         assertEquals(urls[j], link.toString());
@@ -768,7 +900,8 @@ public class DataStoreTestUtil {
 
       assertNotNull(page.getUrl());
       assertEquals(page.getUrl().toString(), SORTED_URLS[i]);
-      assertEquals(0, page.getOutlinks().size());
+      assertNull("Map of Outlinks should be 'null' as the deleteByQuery "
+          + "not only removes the data but also the data structure.", page.getOutlinks());
       assertEquals(0, page.getParsedContent().size());
       if(page.getContent() != null) {
         System.out.println("url:" + page.getUrl().toString());

Modified: gora/branches/GORA_94/gora-hbase/src/main/java/org/apache/gora/hbase/store/HBaseStore.java
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-hbase/src/main/java/org/apache/gora/hbase/store/HBaseStore.java?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-hbase/src/main/java/org/apache/gora/hbase/store/HBaseStore.java (original)
+++ gora/branches/GORA_94/gora-hbase/src/main/java/org/apache/gora/hbase/store/HBaseStore.java Tue Dec  3 21:27:54 2013
@@ -396,8 +396,9 @@ implements Configurable {
         K endKey = Arrays.equals(HConstants.EMPTY_END_ROW, splitStop) ?
             null : HBaseByteInterface.fromBytes(keyClass, splitStop);
 
-        PartitionQuery<K, T> partition = new PartitionQueryImpl<K, T>(
+        PartitionQueryImpl<K, T> partition = new PartitionQueryImpl<K, T>(
             query, startKey, endKey, regionLocation);
+        partition.setConf(getConf());
 
         partitions.add(partition);
       }

Modified: gora/branches/GORA_94/gora-hbase/src/test/java/org/apache/gora/hbase/store/TestHBaseStore.java
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-hbase/src/test/java/org/apache/gora/hbase/store/TestHBaseStore.java?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-hbase/src/test/java/org/apache/gora/hbase/store/TestHBaseStore.java (original)
+++ gora/branches/GORA_94/gora-hbase/src/test/java/org/apache/gora/hbase/store/TestHBaseStore.java Tue Dec  3 21:27:54 2013
@@ -137,7 +137,8 @@ public class TestHBaseStore extends Data
     assertNull(actualBytes);
     table.close();
     
-    // Test writing+reading an empty bytes field. FIELD in HBASE MUST become EMPTY (byte[0])
+    // Test writing+reading an empty bytes field. FIELD in HBASE MUST 
+    // become EMPTY (byte[0])
     page = webPageStore.get("com.example/http") ;
     page.setContent(ByteBuffer.wrap("".getBytes())) ;
     webPageStore.put("com.example/http", page) ;
@@ -156,7 +157,8 @@ public class TestHBaseStore extends Data
   }
   
   /**
-   * Checks that when writing a top level union <code>['null','type']</code> the value is written in raw format
+   * Checks that when writing a top level union <code>['null','type']</code> 
+   * the value is written in raw format
    * @throws Exception
    */
   @Test
@@ -184,8 +186,10 @@ public class TestHBaseStore extends Data
   }
   
   /**
-   * Checks that when writing a top level union <code>['null','type']</code> with the option <code>RAW_ROOT_FIELDS_OPTION=true</code>
-   * the column is not created, and when <code>RAW_ROOT_FIELDS_OPTION=false</code> the <code>null</code> value is serialized
+   * Checks that when writing a top level union <code>['null','type']</code> 
+   * with the option <code>RAW_ROOT_FIELDS_OPTION=true</code>
+   * the column is not created, and when <code>RAW_ROOT_FIELDS_OPTION=false</code> 
+   * the <code>null</code> value is serialized
    * with Avro.
    * @throws Exception
    */

Modified: gora/branches/GORA_94/gora-solr/src/main/java/org/apache/gora/solr/store/SolrStore.java
URL: http://svn.apache.org/viewvc/gora/branches/GORA_94/gora-solr/src/main/java/org/apache/gora/solr/store/SolrStore.java?rev=1547591&r1=1547590&r2=1547591&view=diff
==============================================================================
--- gora/branches/GORA_94/gora-solr/src/main/java/org/apache/gora/solr/store/SolrStore.java (original)
+++ gora/branches/GORA_94/gora-solr/src/main/java/org/apache/gora/solr/store/SolrStore.java Tue Dec  3 21:27:54 2013
@@ -471,7 +471,9 @@ public class SolrStore<K, T extends Pers
     // TODO: implement this using Hadoop DB support
 
     ArrayList<PartitionQuery<K, T>> partitions = new ArrayList<PartitionQuery<K, T>>();
-    partitions.add( new PartitionQueryImpl<K, T>( query ) );
+    PartitionQueryImpl<K, T> pqi = new PartitionQueryImpl<K, T>(query);
+    pqi.setConf(getConf());
+    partitions.add(pqi);
 
     return partitions;
   }