You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gora.apache.org by rm...@apache.org on 2014/04/12 21:21:56 UTC

svn commit: r1586888 [6/10] - in /gora/trunk: ./ bin/ gora-accumulo/ gora-accumulo/src/main/java/org/apache/gora/accumulo/encoders/ gora-accumulo/src/main/java/org/apache/gora/accumulo/query/ gora-accumulo/src/main/java/org/apache/gora/accumulo/store/ ...

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/Persistent.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/Persistent.java?rev=1586888&r1=1586887&r2=1586888&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/Persistent.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/Persistent.java Sat Apr 12 19:21:53 2014
@@ -1,185 +1,119 @@
 /**
- * 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.
- */
+* 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.persistency;
 
-/**
- * Objects that are persisted by Gora implements this interface.
- */
-public interface Persistent extends Cloneable{
+import java.util.List;
 
-  /**
-   * Returns the StateManager which manages the persistent 
-   * state of the object.
-   * @return the StateManager of the object
-   */
-  StateManager getStateManager();
+import org.apache.avro.Schema.Field;
+import org.apache.avro.specific.SpecificRecord;
 
-  /**
-   * Constructs a new instance of the object with the given StateManager.
-   * This method is intended to be used by Gora framework.
-   * @param stateManager the StateManager to manage the persistent state 
-   * of the object
-   * @return a new instance of the object
-   */
-  Persistent newInstance(StateManager stateManager);
+import org.apache.gora.persistency.Dirtyable;
+
+/**
+* Objects that are persisted by Gora implements this interface.
+*/
+public interface Persistent extends SpecificRecord, Dirtyable {
+
+  public static String DIRTY_BYTES_FIELD_NAME = "__g__dirty";
 
   /**
-   * Returns sorted field names of the object
-   * @return the field names of the object as a String[]
-   */
-  String[] getFields();
-  
-  /**
-   * Returns the field name with the given index
-   * @param index the index of the field  
-   * @return the name of the field
-   */
-  String getField(int index);
-  
-  /**
-   * Returns the index of the field with the given name
-   * @param field the name of the field
-   * @return the index of the field
-   */
-  int getFieldIndex(String field);
-  
-  /**
-   * Clears the inner state of the object without any modification
-   * to the actual data on the data store. This method should be called 
-   * before re-using the object to hold the data for another result.  
-   */
+* Clears the inner state of the object without any modification to the actual
+* data on the data store. This method should be called before re-using the
+* object to hold the data for another result.
+*/
   void clear();
-  
-  /**
-   * Returns whether the object is newly constructed.
-   * @return true if the object is newly constructed, false if
-   * retrieved from a datastore. 
-   */
-  boolean isNew();
-  
-  /**
-   * Sets the state of the object as new for persistency
-   */
-  void setNew();
-  
-  /**
-   * Clears the new state 
-   */
-  void clearNew();
-  
-  /**
-   * Returns whether any of the fields of the object has been modified 
-   * after construction or loading. 
-   * @return whether any of the fields of the object has changed
-   */
-  boolean isDirty();
-  
+
   /**
-   * Returns whether the field has been modified.
-   * @param fieldIndex the offset of the field in the object
-   * @return whether the field has been modified.
-   */
+* Returns whether the field has been modified.
+*
+* @param fieldIndex
+* the offset of the field in the object
+* @return whether the field has been modified.
+*/
   boolean isDirty(int fieldIndex);
 
   /**
-   * Returns whether the field has been modified.
-   * @param field the name of the field
-   * @return whether the field has been modified.
-   */
+* Returns whether the field has been modified.
+*
+* @param field
+* the name of the field
+* @return whether the field has been modified.
+*/
   boolean isDirty(String field);
-  
+
   /**
-   * Sets all the fields of the object as dirty.
-   */
+* Sets all the fields of the object as dirty.
+*/
   void setDirty();
-  
+
   /**
-   * Sets the field as dirty.
-   * @param fieldIndex the offset of the field in the object
-   */
+* Sets the field as dirty.
+*
+* @param fieldIndex
+* the offset of the field in the object
+*/
   void setDirty(int fieldIndex);
- 
+
   /**
-   * Sets the field as dirty.
-   * @param field the name of the field
-   */
+* Sets the field as dirty.
+*
+* @param field
+* the name of the field
+*/
   void setDirty(String field);
-  
-  /**
-   * Clears the field as dirty.
-   * @param fieldIndex the offset of the field in the object
-   */
-  void clearDirty(int fieldIndex);
-  
-  /**
-   * Clears the field as dirty.
-   * @param field the name of the field
-   */
-  void clearDirty(String field);
-  
-  /**
-   * Clears the dirty state.
-   */
-  void clearDirty();
-  
-  /**
-   * Returns whether the field has been loaded from the datastore. 
-   * @param fieldIndex the offset of the field in the object
-   * @return whether the field has been loaded 
-   */
-  boolean isReadable(int fieldIndex);
 
   /**
-   * Returns whether the field has been loaded from the datastore. 
-   * @param field the name of the field
-   * @return whether the field has been loaded 
-   */
-  boolean isReadable(String field);
-  
-  /**
-   * Sets the field as readable.
-   * @param fieldIndex the offset of the field in the object
-   */
-  void setReadable(int fieldIndex);
+* Clears the field as dirty.
+*
+* @param fieldIndex
+* the offset of the field in the object
+*/
+  void clearDirty(int fieldIndex);
 
   /**
-   * Sets the field as readable.
-   * @param field the name of the field
-   */
-  void setReadable(String field);
+* Clears the field as dirty.
+*
+* @param field
+* the name of the field
+*/
+  void clearDirty(String field);
 
   /**
-   * Clears the field as readable.
-   * @param fieldIndex the offset of the field in the object
-   */
-  void clearReadable(int fieldIndex);
-  
-  /**
-   * Sets the field as readable.
-   * @param field the name of the field
-   */
-  void clearReadable(String field);
-  
-  /**
-   * Clears the readable state.
+* Get an object which can be used to mark this field as deleted (rather than
+* state unknown, which is indicated by null).
+*
+* @return a tombstone.
+*/
+  public abstract Tombstone getTombstone();
+
+  /**
+* Get a list of fields from this persistent object's schema that are not
+* managed by Gora.
+*
+* @return the unmanaged fields
+*/
+  public List<Field> getUnmanagedFields();
+
+  /**
+   * Constructs a new instance of the object by using appropriate builder. This
+   * method is intended to be used by Gora framework.
+   * 
+   * @return a new instance of the object
    */
-  void clearReadable();
-  
-  Persistent clone();
-
+  Persistent newInstance();
 }
\ No newline at end of file

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/State.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/State.java?rev=1586888&r1=1586887&r2=1586888&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/State.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/State.java Sat Apr 12 19:21:53 2014
@@ -1,37 +0,0 @@
-/**
- * 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.persistency;
-
-/**
- * Persistency state of an object or field.
- */
-public enum State {
-  
-  /** The object is newly loaded */
-  NEW,
-  
-  /** The value of the field has not been changed after loading*/
-  CLEAN,
-  
-  /** The value of the field has been altered*/
-  DIRTY,
-  
-  /** The object or field is deleted */
-  DELETED
-}

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/StateManager.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/StateManager.java?rev=1586888&r1=1586887&r2=1586888&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/StateManager.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/StateManager.java Sat Apr 12 19:21:53 2014
@@ -1,110 +0,0 @@
-/**
- * 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.persistency;
-
-/**
- * StateManager manages objects state for persistency.
- */
-public interface StateManager{
-
-  /**
-   * If one state manager is allocated per persistent object, 
-   * call this method to set the managed persistent. 
-   * @param persistent the persistent to manage
-   */
-  void setManagedPersistent(Persistent persistent);
-
-  /**
-   * Returns whether the object is newly constructed.
-   * @return true if the object is newly constructed, false if
-   * retrieved from a datastore. 
-   */
-  boolean isNew(Persistent persistent);
-  
-  /**
-   * Sets the state of the object as new for persistency
-   */
-  void setNew(Persistent persistent);
-  
-  /**
-   * Clears the new state 
-   */
-  void clearNew(Persistent persistent);
-
-  /**
-   * Returns whether any of the fields of the object has been modified 
-   * after construction or loading. 
-   * @return whether any of the fields of the object has changed
-   */
-  boolean isDirty(Persistent persistent);
-  
-  /**
-   * Returns whether the field has been modified.
-   * @param fieldIndex the offset of the field in the object
-   * @return whether the field has been modified.
-   */
-  boolean isDirty(Persistent persistent, int fieldIndex);
-  
-  /**
-   * Sets all the fields of the object as dirty.
-   */
-  void setDirty(Persistent persistent);
-  
-  /**
-   * Sets the field as dirty.
-   * @param fieldIndex the offset of the field in the object
-   */
-  void setDirty(Persistent persistent, int fieldIndex);
-
-  /**
-   * Clears the field as dirty.
-   * @param fieldIndex the offset of the field in the object
-   */
-  void clearDirty(Persistent persistent, int fieldIndex);
-  
-  /**
-   * Clears the dirty state.
-   */
-  void clearDirty(Persistent persistent);
-  
-  /**
-   * Returns whether the field has been loaded from the datastore. 
-   * @param fieldIndex the offset of the field in the object
-   * @return whether the field has been loaded 
-   */
-  boolean isReadable(Persistent persistent, int fieldIndex);
-  
-  /**
-   * Sets the field as readable.
-   * @param fieldIndex the offset of the field in the object
-   */
-  void setReadable(Persistent persistent, int fieldIndex);
-
-  /**
-   * Clears the field as readable.
-   * @param fieldIndex the offset of the field in the object
-   */
-  void clearReadable(Persistent persistent, int fieldIndex);
-  
-  /**
-   * Clears the readable state.
-   */
-  void clearReadable(Persistent persistent);
-  
-}

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/StatefulHashMap.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/StatefulHashMap.java?rev=1586888&r1=1586887&r2=1586888&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/StatefulHashMap.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/StatefulHashMap.java Sat Apr 12 19:21:53 2014
@@ -1,132 +0,0 @@
-/**
- * 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.persistency;
-
-import java.util.HashMap;
-import java.util.Map;
-
-@SuppressWarnings("serial")
-public class StatefulHashMap<K, V> extends HashMap<K, V> 
-  implements StatefulMap<K, V> {
-  
-  /* This is probably a terrible design but I do not yet have a better
-   * idea of managing write/delete info on a per-key basis
-   */
-  private Map<K, State> keyStates = new HashMap<K, State>();
-
-  /**
-   * Create an empty instance.
-   */
-  public StatefulHashMap() {
-    this(null);
-  }
-
-  /**
-   * Create an instance with initial entries. These entries are added stateless;
-   * in other words the statemap will be clear after the construction.
-   * 
-   * @param m The map with initial entries.
-   */
-  public StatefulHashMap(Map<K, V> m) {
-    super();
-    if (m == null) {
-      return;
-    }
-    for (java.util.Map.Entry<K, V> entry : m.entrySet()) {
-      put(entry.getKey(), entry.getValue());
-    }
-    clearStates();
-  }
-  
-  @Override
-  public V put(K key, V value) {
-    V old = super.put(key, value);
-    //if old value is different or null, set state to dirty
-    if (!value.equals(old)) {
-      keyStates.remove(key);
-      keyStates.put(key, State.DIRTY);
-    }
-    return old;
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public V remove(Object key) {
-    keyStates.put((K) key, State.DELETED);
-    return null;
-    // We do not remove the actual entry from the map.
-    // When we keep the entries, we can compare previous state to make Datastore
-    // puts more efficient. (In the case of new puts that are in fact unchanged)
-  }
-
-  @Override
-  public void putAll(Map<? extends K, ? extends V> m) {
-    for (Entry<? extends K, ? extends V> e : m.entrySet()) {
-      put(e.getKey(), e.getValue());
-    }
-  }
-
-  @Override
-  public void clear() {
-    // The problem with clear() is that we cannot delete entries that were not
-    // initially set on the input.  This means that for a clear() to fully
-    // reflect on a datastore you have to input the full map from the store.
-    // This is acceptable for now. Another way around this is to implement
-    // some sort of "clear marker" that indicates a map should be fully cleared,
-    // with respect to any possible new entries.
-    for (Entry<K, V> e : entrySet()) {
-      keyStates.put(e.getKey(), State.DELETED);
-    }
-    // Do not actually clear the map, i.e. with super.clear()
-    // When we keep the entries, we can compare previous state to make Datastore
-    // puts more efficient. (In the case of new puts that are in fact unchanged)
-  }
-
-  public State getState(K key) {
-    return keyStates.get(key);
-  };
-  
-  /* (non-Javadoc)
-   * @see org.apache.gora.persistency.StatefulMap#resetStates()
-   */
-  public void clearStates() {
-    keyStates.clear();
-  }
-
-  /* (non-Javadoc)
-   * @see org.apache.gora.persistency.StatefulMap#putState(K, org.apache.gora.persistency.State)
-   */
-  public void putState(K key, State state) {
-    keyStates.put(key, state);
-  }
-
-  /* (non-Javadoc)
-   * @see org.apache.gora.persistency.StatefulMap#states()
-   */
-  public Map<K, State> states() {
-    return keyStates;
-  }
-
-  /* (non-Javadoc)
-   * @see org.apache.gora.persistency.StatefulMap#reuse()
-   */
-  public void reuse() {
-    super.clear();
-    clearStates();
-  }
-}

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/StatefulMap.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/StatefulMap.java?rev=1586888&r1=1586887&r2=1586888&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/StatefulMap.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/StatefulMap.java Sat Apr 12 19:21:53 2014
@@ -1,43 +0,0 @@
-/**
- * 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.persistency;
-
-import java.util.Map;
-
-/**
- * StatefulMap extends the Map interface to keep track of the 
- * persistency states of individual elements in the Map.  
- */
-public interface StatefulMap<K, V> extends Map<K, V> {
-
-  State getState(K key);
-  
-  void putState(K key, State state);
-
-  Map<K, State> states();
-
-  void clearStates();
-  
-  /**
-   * Reuse will clear the map completely with states. This is different
-   * from {@link #clear()} in that the latter only sets entries to deleted.
-   */
-  void reuse();
-  
-}
\ No newline at end of file

Added: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/Tombstone.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/Tombstone.java?rev=1586888&view=auto
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/Tombstone.java (added)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/Tombstone.java Sat Apr 12 19:21:53 2014
@@ -0,0 +1,16 @@
+package org.apache.gora.persistency;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Marker interface used to signal a deletion to Gora.
+ */
+public interface Tombstone {
+
+}

Propchange: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/Tombstone.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/Tombstones.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/Tombstones.java?rev=1586888&view=auto
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/Tombstones.java (added)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/Tombstones.java Sat Apr 12 19:21:53 2014
@@ -0,0 +1,242 @@
+/**
+* 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.persistency;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Set;
+
+public final class Tombstones {
+
+  private Tombstones() {
+
+  }
+
+  public static final class MapTombstone<K, V> implements Tombstone, Map<K, V> {
+
+    private MapTombstone() {
+    }
+
+    private static final Map DELEGATE = Collections.EMPTY_MAP;
+
+    public int size() {
+      return DELEGATE.size();
+    }
+
+    public boolean isEmpty() {
+      return DELEGATE.isEmpty();
+    }
+
+    public boolean containsKey(Object key) {
+      return DELEGATE.containsKey(key);
+    }
+
+    public boolean containsValue(Object value) {
+      return DELEGATE.containsValue(value);
+    }
+
+    @SuppressWarnings("unchecked")
+    public V get(Object key) {
+      return (V) DELEGATE.get(key);
+    }
+
+    @SuppressWarnings("unchecked")
+    public V put(Object key, Object value) {
+      return (V) DELEGATE.put(key, value);
+    }
+
+    @SuppressWarnings("unchecked")
+    public V remove(Object key) {
+      return (V) DELEGATE.remove(key);
+    }
+
+    @SuppressWarnings("unchecked")
+    public void putAll(Map m) {
+      DELEGATE.putAll(m);
+    }
+
+    public void clear() {
+      DELEGATE.clear();
+    }
+
+    @SuppressWarnings("unchecked")
+    public Set keySet() {
+      return DELEGATE.keySet();
+    }
+
+    @SuppressWarnings("unchecked")
+    public Collection values() {
+      return DELEGATE.values();
+    }
+
+    @SuppressWarnings("unchecked")
+    public Set entrySet() {
+      return DELEGATE.entrySet();
+    }
+
+    public boolean equals(Object o) {
+      return DELEGATE.equals(o);
+    }
+
+    public int hashCode() {
+      return DELEGATE.hashCode();
+    }
+
+  }
+
+  public static final class ListTombstone<T> implements List<T>, Tombstone {
+    
+    private static final List DELEGATE = Collections.EMPTY_LIST;
+
+    private ListTombstone() {
+    }
+
+    public int size() {
+      return DELEGATE.size();
+    }
+
+    public boolean isEmpty() {
+      return DELEGATE.isEmpty();
+    }
+
+    public boolean contains(Object o) {
+      return DELEGATE.contains(o);
+    }
+
+    public Iterator iterator() {
+      return DELEGATE.iterator();
+    }
+
+    public Object[] toArray() {
+      return DELEGATE.toArray();
+    }
+
+    @SuppressWarnings("unchecked")
+    public Object[] toArray(Object[] a) {
+      return DELEGATE.toArray(a);
+    }
+
+    @SuppressWarnings("unchecked")
+    public boolean add(Object e) {
+      return DELEGATE.add(e);
+    }
+
+    public boolean remove(Object o) {
+      return DELEGATE.remove(o);
+    }
+
+    @SuppressWarnings("unchecked")
+    public boolean containsAll(Collection c) {
+      return DELEGATE.containsAll(c);
+    }
+
+    @SuppressWarnings("unchecked")
+    public boolean addAll(Collection c) {
+      return DELEGATE.addAll(c);
+    }
+
+    @SuppressWarnings("unchecked")
+    public boolean addAll(int index, Collection c) {
+      return DELEGATE.addAll(index, c);
+    }
+
+    @SuppressWarnings("unchecked")
+    public boolean removeAll(Collection c) {
+      return DELEGATE.removeAll(c);
+    }
+
+    @SuppressWarnings("unchecked")
+    public boolean retainAll(Collection c) {
+      return DELEGATE.retainAll(c);
+    }
+
+    public void clear() {
+      DELEGATE.clear();
+    }
+
+    public boolean equals(Object o) {
+      return DELEGATE.equals(o);
+    }
+
+    public int hashCode() {
+      return DELEGATE.hashCode();
+    }
+
+    @SuppressWarnings("unchecked")
+    public T get(int index) {
+      return (T) DELEGATE.get(index);
+    }
+
+    @SuppressWarnings("unchecked")
+    public Object set(int index, Object element) {
+      return DELEGATE.set(index, element);
+    }
+
+    @SuppressWarnings("unchecked")
+    public void add(int index, Object element) {
+      DELEGATE.add(index, element);
+    }
+
+    @SuppressWarnings("unchecked")
+    public T remove(int index) {
+      return (T) DELEGATE.remove(index);
+    }
+
+    public int indexOf(Object o) {
+      return DELEGATE.indexOf(o);
+    }
+
+    public int lastIndexOf(Object o) {
+      return DELEGATE.lastIndexOf(o);
+    }
+
+    public ListIterator listIterator() {
+      return DELEGATE.listIterator();
+    }
+
+    public ListIterator listIterator(int index) {
+      return DELEGATE.listIterator(index);
+    }
+
+    public List subList(int fromIndex, int toIndex) {
+      return DELEGATE.subList(fromIndex, toIndex);
+    }
+
+  }
+
+  public static final MapTombstone MAP_TOMBSTONE = new MapTombstone();
+
+  public static <K, V> MapTombstone<K, V> getMapTombstone() {
+    return MAP_TOMBSTONE;
+  }
+
+  public static final ListTombstone LIST_TOMBSTONE = new ListTombstone();
+
+  public static final <T> ListTombstone<T> getListTombstone() {
+    return LIST_TOMBSTONE;
+  }
+
+  public static boolean isTombstone(Object o) {
+    return (o instanceof Tombstone);
+  }
+
+}

Propchange: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/Tombstones.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/BeanFactoryImpl.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/BeanFactoryImpl.java?rev=1586888&r1=1586887&r2=1586888&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/BeanFactoryImpl.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/BeanFactoryImpl.java Sat Apr 12 19:21:53 2014
@@ -31,16 +31,29 @@ import org.apache.gora.util.ReflectionUt
  */
 public class BeanFactoryImpl<K, T extends Persistent> implements BeanFactory<K, T> {
 
+  /** Class of the key to be used */
   private Class<K> keyClass;
+  
+  /** Class of the persistent objects to be stored */
   private Class<T> persistentClass;
   
+  /** Constructor of the key */
   private Constructor<K> keyConstructor;
   
+  /** Object's key */
   private K key;
+  
+  /** Persistent object of class T */
   private T persistent;
   
+  /** Flag to be used to determine if a key is persistent or not */
   private boolean isKeyPersistent = false;
   
+  /**
+   * Default constructor for this class.
+   * @param keyClass.
+   * @param persistentClass
+   */
   public BeanFactoryImpl(Class<K> keyClass, Class<T> persistentClass) {
     this.keyClass = keyClass;
     this.persistentClass = persistentClass;
@@ -59,21 +72,13 @@ public class BeanFactoryImpl<K, T extend
   }
   
   @Override
-  @SuppressWarnings("unchecked")
   public K newKey() throws Exception {
-    if(isKeyPersistent)
-      return (K)((Persistent)key).newInstance(new StateManagerImpl());
-    else if(keyConstructor == null) {
-      throw new RuntimeException("Key class does not have a no-arg constructor");
-    }
-    else
-      return keyConstructor.newInstance(ReflectionUtils.EMPTY_OBJECT_ARRAY);
+    return keyClass.newInstance();
   }
  
-  @SuppressWarnings("unchecked")
   @Override
   public T newPersistent() {
-    return (T) persistent.newInstance(new StateManagerImpl());
+    return (T) persistent.newInstance();
   }
   
   @Override

Added: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyCollectionWrapper.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyCollectionWrapper.java?rev=1586888&view=auto
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyCollectionWrapper.java (added)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyCollectionWrapper.java Sat Apr 12 19:21:53 2014
@@ -0,0 +1,145 @@
+package org.apache.gora.persistency.impl;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.gora.persistency.Dirtyable;
+
+/**
+ * A {@link List} implementation that wraps another list, intercepting
+ * modifications to the list structure and reporting on weather or not the list
+ * has been modified, and also checking list elements for modification.
+ * 
+ * @param <T>
+ *          The type of the list that this wrapper wraps.
+ */
+public class DirtyCollectionWrapper<T> implements Dirtyable,
+    Collection<T> {
+
+  /** The delegate list that the wrapper wraps */
+  private final Collection<T> delegate;
+  /**
+   * The dirty flag, tracks if the structure of the underlying list has been
+   * modified
+   */
+  private DirtyFlag dirtyFlag;
+
+  DirtyCollectionWrapper(Collection<T> delegate2, DirtyFlag dirtyFlag) {
+    this.delegate = delegate2;
+    this.dirtyFlag = dirtyFlag;
+  }
+
+  @Override
+  public boolean isDirty() {
+    boolean anyDirty = false;
+    for (T value : this) {
+      anyDirty = anyDirty || (value instanceof Dirtyable) ? ((Dirtyable) value).isDirty():false;
+    }
+    return anyDirty || dirtyFlag.isDirty();
+  }
+
+  @Override
+  public void clearDirty() {
+    for (T value : this) {
+      if (value instanceof Dirtyable)
+        ((Dirtyable) value).clearDirty();
+    }
+    dirtyFlag.clearDirty();
+  }
+
+  @Override
+  public int size() {
+    return delegate.size();
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return delegate.isEmpty();
+  }
+
+  @Override
+  public boolean contains(Object o) {
+    return delegate.contains(o);
+  }
+
+  @Override
+  public Iterator<T> iterator() {
+    return new DirtyIteratorWrapper<T>(delegate.iterator(), dirtyFlag);
+  }
+
+  @Override
+  public Object[] toArray() {
+    return delegate.toArray();
+  }
+
+  @Override
+  public <R> R[] toArray(R[] a) {
+    return delegate.toArray(a);
+  }
+
+  @Override
+  public boolean add(T e) {
+    boolean change = delegate.add(e);
+    dirtyFlag.makeDirty(change);
+    return change;
+  }
+
+  @Override
+  public boolean remove(Object o) {
+    boolean change = delegate.remove(o);
+    dirtyFlag.makeDirty(change);
+    return change;
+  }
+
+  @Override
+  public boolean containsAll(Collection<?> c) {
+    return delegate.containsAll(c);
+  }
+
+  @Override
+  public boolean addAll(Collection<? extends T> c) {
+    boolean change = delegate.addAll(c);
+    dirtyFlag.makeDirty(change);
+    return change;
+  }
+
+  @Override
+  public boolean removeAll(Collection<?> c) {
+    boolean change = delegate.removeAll(c);
+    dirtyFlag.makeDirty(change);
+    return change;
+  }
+
+  @Override
+  public boolean retainAll(Collection<?> c) {
+    boolean change = delegate.retainAll(c);
+    dirtyFlag.makeDirty(change);
+    return change;
+  }
+
+  @Override
+  public void clear() {
+    dirtyFlag.makeDirty(size() > 0);
+    delegate.clear();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    return delegate.equals(o);
+  }
+
+  @Override
+  public int hashCode() {
+    return delegate.hashCode();
+  }
+
+  protected Collection<T> getDelegate() {
+    return delegate;
+  }
+
+  protected DirtyFlag getDirtyFlag() {
+    return dirtyFlag;
+  }
+
+}

Propchange: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyCollectionWrapper.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyFlag.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyFlag.java?rev=1586888&view=auto
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyFlag.java (added)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyFlag.java Sat Apr 12 19:21:53 2014
@@ -0,0 +1,56 @@
+/**
+ * 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.persistency.impl;
+
+import org.apache.gora.persistency.Dirtyable;
+
+final class DirtyFlag implements Dirtyable {
+
+  private boolean dirty;
+
+  public DirtyFlag(boolean dirty) {
+    this.dirty = dirty;
+  }
+
+  public DirtyFlag() {
+    this.dirty = false;
+  }
+
+  @Override
+  public boolean isDirty() {
+    return dirty;
+  }
+
+  @Override
+  public void clearDirty() {
+    this.dirty = false;
+  }
+
+  /**
+   * Set this DirtyFlag to dirty if the <tt>dirty</tt> operand is true. If
+   * not, the state of the flag remains unchanged.
+   * 
+   * @param dirty
+   *            Weather or not to set this flag to dirty. If false, the state
+   *            is unchanged.
+   */
+  public void makeDirty(boolean dirty) {
+    this.dirty = this.dirty || dirty;
+  }
+
+}

Propchange: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyFlag.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyIteratorWrapper.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyIteratorWrapper.java?rev=1586888&view=auto
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyIteratorWrapper.java (added)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyIteratorWrapper.java Sat Apr 12 19:21:53 2014
@@ -0,0 +1,35 @@
+package org.apache.gora.persistency.impl;
+
+import java.util.Iterator;
+
+/**
+ * Sets the dirty flag if the iterator's remove method is called.
+ */
+final class DirtyIteratorWrapper<T> implements Iterator<T> {
+
+  private final DirtyFlag dirtyFlag;
+  private Iterator<T> delegateIterator;
+
+  DirtyIteratorWrapper(Iterator<T> delegateIterator,
+      DirtyFlag dirtyFlag) {
+    this.delegateIterator = delegateIterator;
+    this.dirtyFlag = dirtyFlag;
+  }
+
+  @Override
+  public boolean hasNext() {
+    return delegateIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    return delegateIterator.next();
+  }
+
+  @Override
+  public void remove() {
+    dirtyFlag.makeDirty(true);
+    delegateIterator.remove();
+  }
+
+}
\ No newline at end of file

Propchange: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyIteratorWrapper.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyListIterator.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyListIterator.java?rev=1586888&view=auto
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyListIterator.java (added)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyListIterator.java Sat Apr 12 19:21:53 2014
@@ -0,0 +1,67 @@
+package org.apache.gora.persistency.impl;
+
+import java.util.ListIterator;
+
+/**
+ * Sets the dirty flag if the list iterator's modification methods (remove,
+ * set, add) are called.
+ */
+final class DirtyListIterator<T> implements ListIterator<T> {
+
+  private final ListIterator<T> iteratorDelegate;
+  private final DirtyFlag dirtyFlag;
+
+  DirtyListIterator(ListIterator<T> delegate, DirtyFlag dirtyFlag) {
+    this.iteratorDelegate = delegate;
+    this.dirtyFlag = dirtyFlag;
+  }
+
+  @Override
+  public boolean hasNext() {
+    return iteratorDelegate.hasNext();
+  }
+
+  @Override
+  public T next() {
+    return iteratorDelegate.next();
+  }
+
+  @Override
+  public boolean hasPrevious() {
+    return iteratorDelegate.hasPrevious();
+  }
+
+  @Override
+  public T previous() {
+    return iteratorDelegate.previous();
+  }
+
+  @Override
+  public int nextIndex() {
+    return iteratorDelegate.nextIndex();
+  }
+
+  @Override
+  public int previousIndex() {
+    return iteratorDelegate.previousIndex();
+  }
+
+  @Override
+  public void remove() {
+    dirtyFlag.makeDirty(true);
+    iteratorDelegate.remove();
+  }
+
+  @Override
+  public void set(T e) {
+    dirtyFlag.makeDirty(true);
+    iteratorDelegate.set(e);
+  }
+
+  @Override
+  public void add(T e) {
+    dirtyFlag.makeDirty(true);
+    iteratorDelegate.add(e);
+  }
+
+}
\ No newline at end of file

Propchange: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyListIterator.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyListWrapper.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyListWrapper.java?rev=1586888&view=auto
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyListWrapper.java (added)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyListWrapper.java Sat Apr 12 19:21:53 2014
@@ -0,0 +1,97 @@
+package org.apache.gora.persistency.impl;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.ListIterator;
+
+import org.apache.gora.persistency.Dirtyable;
+
+/**
+ * A {@link List} implementation that wraps another list, intercepting
+ * modifications to the list structure and reporting on weather or not the list
+ * has been modified, and also checking list elements for modification.
+ * 
+ * @param <T>
+ *          The type of the list that this wrapper wraps.
+ */
+public class DirtyListWrapper<T> extends DirtyCollectionWrapper<T> implements
+    Dirtyable, List<T> {
+
+  /**
+   * Create a DirtyListWrapper that wraps a getDelegate().
+   * 
+   * @param delegate
+   *          The getDelegate().to wrap.
+   */
+  public DirtyListWrapper(List<T> delegate) {
+    this(delegate, new DirtyFlag());
+  }
+
+  DirtyListWrapper(List<T> delegate, DirtyFlag dirtyFlag) {
+    super(delegate, dirtyFlag);
+  }
+
+  @Override
+  public boolean addAll(int index, Collection<? extends T> c) {
+    boolean change = getDelegate().addAll(index, c);
+    getDirtyFlag().makeDirty(change);
+    return change;
+  }
+
+  @Override
+  public T get(int index) {
+    return getDelegate().get(index);
+  }
+
+  @Override
+  public T set(int index, T element) {
+    getDirtyFlag().makeDirty(true);
+    return getDelegate().set(index, element);
+  }
+
+  @Override
+  public void add(int index, T element) {
+    getDirtyFlag().makeDirty(true);
+    getDelegate().add(index, element);
+  }
+
+  @Override
+  public T remove(int index) {
+    getDirtyFlag().makeDirty(true);
+    return getDelegate().remove(index);
+  }
+
+  @Override
+  public int indexOf(Object o) {
+    return getDelegate().indexOf(o);
+  }
+
+  @Override
+  public int lastIndexOf(Object o) {
+    return getDelegate().lastIndexOf(o);
+  }
+
+  @Override
+  public ListIterator<T> listIterator() {
+    return new DirtyListIterator<T>(getDelegate().listIterator(),
+        getDirtyFlag());
+  }
+
+  @Override
+  public ListIterator<T> listIterator(int index) {
+    return new DirtyListIterator<T>(getDelegate().listIterator(index),
+        getDirtyFlag());
+  }
+
+  @Override
+  public List<T> subList(int fromIndex, int toIndex) {
+    return new DirtyListWrapper<T>(getDelegate().subList(fromIndex, toIndex),
+        getDirtyFlag());
+  }
+
+  @Override
+  protected List<T> getDelegate() {
+    return (List<T>) super.getDelegate();
+  }
+
+}

Propchange: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyListWrapper.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyMapWrapper.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyMapWrapper.java?rev=1586888&view=auto
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyMapWrapper.java (added)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyMapWrapper.java Sat Apr 12 19:21:53 2014
@@ -0,0 +1,194 @@
+package org.apache.gora.persistency.impl;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.gora.persistency.Dirtyable;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Collections2;
+
+public class DirtyMapWrapper<K, V> implements Map<K, V>, Dirtyable {
+
+  public static class DirtyEntryWrapper<K, V> implements Entry<K, V>, Dirtyable {
+    private final Entry<K, V> entryDelegate;
+    private DirtyFlag dirtyFlag;
+
+    public DirtyEntryWrapper(Entry<K, V> delegate, DirtyFlag dirtyFlag) {
+      this.entryDelegate = delegate;
+      this.dirtyFlag = dirtyFlag;
+    }
+
+    @Override
+    public K getKey() {
+      return entryDelegate.getKey();
+    }
+
+    @Override
+    public V getValue() {
+      return entryDelegate.getValue();
+    }
+
+    @Override
+    public V setValue(V value) {
+      dirtyFlag.makeDirty(valueChanged(value, entryDelegate.getValue()));
+      return entryDelegate.setValue(value);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      return entryDelegate.equals(o);
+    }
+
+    @Override
+    public int hashCode() {
+      return entryDelegate.hashCode();
+    }
+
+    @Override
+    public boolean isDirty() {
+      return dirtyFlag.isDirty() || (entryDelegate instanceof Dirtyable) ? ((Dirtyable) entryDelegate
+          .getValue()).isDirty() : false;
+    }
+
+    @Override
+    public void clearDirty() {
+      dirtyFlag.clearDirty();
+    }
+
+  }
+
+  private final Map<K, V> delegate;
+
+  private final DirtyFlag dirtyFlag;
+
+  public DirtyMapWrapper(Map<K, V> delegate) {
+    this(delegate, new DirtyFlag());
+  }
+
+  DirtyMapWrapper(Map<K, V> delegate, DirtyFlag dirtyFlag) {
+    this.dirtyFlag = dirtyFlag;
+    this.delegate = delegate;
+  }
+
+  @Override
+  public boolean isDirty() {
+    boolean anyDirty = false;
+    for (V v : this.values()) {
+      anyDirty = anyDirty || (v instanceof Dirtyable) ? ((Dirtyable) v)
+          .isDirty() : false;
+    }
+    return anyDirty || dirtyFlag.isDirty();
+  }
+
+  @Override
+  public void clearDirty() {
+    for (V v : this.values()) {
+      if (v instanceof Dirtyable)
+        ((Dirtyable) v).clearDirty();
+    }
+    dirtyFlag.clearDirty();
+  }
+
+  @Override
+  public int size() {
+    return delegate.size();
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return delegate.isEmpty();
+  }
+
+  @Override
+  public boolean containsKey(Object key) {
+    return delegate.containsKey(key);
+  }
+
+  @Override
+  public boolean containsValue(Object value) {
+    return delegate.containsValue(value);
+  }
+
+  @Override
+  public V get(Object key) {
+    return delegate.get(key);
+  }
+
+  @Override
+  public V put(K key, V value) {
+    checkPutWillMakeDirty(key, value);
+    return delegate.put(key, value);
+  }
+
+  private void checkPutWillMakeDirty(K key, V value) {
+    if (containsKey(key)) {
+      dirtyFlag.makeDirty(valueChanged(value, get(key)));
+    } else {
+      dirtyFlag.makeDirty(true);
+    }
+  }
+
+  private static <V> boolean valueChanged(V value, V oldValue) {
+    return (value == null && oldValue != null)
+        || (value != null && !value.equals(oldValue));
+  }
+
+  @Override
+  public V remove(Object key) {
+    dirtyFlag.makeDirty(containsKey(key));
+    return delegate.remove(key);
+  }
+
+  @Override
+  public void putAll(Map<? extends K, ? extends V> m) {
+    for (Entry<? extends K, ? extends V> entry : m.entrySet()) {
+      checkPutWillMakeDirty(entry.getKey(), entry.getValue());
+    }
+    delegate.putAll(m);
+  }
+
+  @Override
+  public void clear() {
+    if (delegate.size() != 0) {
+      dirtyFlag.makeDirty(true);
+    }
+    delegate.clear();
+  }
+
+  @Override
+  public Set<K> keySet() {
+    return delegate.keySet();
+  }
+
+  @Override
+  public Collection<V> values() {
+    return new DirtyCollectionWrapper<V>(delegate.values(), dirtyFlag);
+  }
+
+  @Override
+  @SuppressWarnings({ "unchecked", "rawtypes" })
+  public Set<java.util.Map.Entry<K, V>> entrySet() {
+    Collection<DirtyEntryWrapper<K, V>> dirtyEntrySet = Collections2.transform(
+        delegate.entrySet(),
+        new Function<Entry<K, V>, DirtyEntryWrapper<K, V>>() {
+          @Override
+          public DirtyEntryWrapper<K, V> apply(java.util.Map.Entry<K, V> input) {
+            return new DirtyEntryWrapper<K, V>(input, dirtyFlag);
+          }
+        });
+    return new DirtySetWrapper(dirtyEntrySet, dirtyFlag);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    return delegate.equals(o);
+  }
+
+  @Override
+  public int hashCode() {
+    return delegate.hashCode();
+  }
+
+}

Propchange: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyMapWrapper.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtySetWrapper.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtySetWrapper.java?rev=1586888&view=auto
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtySetWrapper.java (added)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtySetWrapper.java Sat Apr 12 19:21:53 2014
@@ -0,0 +1,15 @@
+package org.apache.gora.persistency.impl;
+
+import java.util.Collection;
+import java.util.Set;
+
+import org.apache.gora.persistency.Dirtyable;
+
+public class DirtySetWrapper<T extends Dirtyable> extends
+    DirtyCollectionWrapper<T> implements Set<T> {
+
+  DirtySetWrapper(Collection<T> delegate2, DirtyFlag dirtyFlag) {
+    super(delegate2, dirtyFlag);
+  }
+
+}

Propchange: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/DirtySetWrapper.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/PersistentBase.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/PersistentBase.java?rev=1586888&r1=1586887&r2=1586888&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/PersistentBase.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/PersistentBase.java Sat Apr 12 19:21:53 2014
@@ -1,327 +1,206 @@
 /**
- * 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.
- */
+* 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.persistency.impl;
 
 import java.nio.ByteBuffer;
-import java.util.HashMap;
+import java.util.Collection;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.avro.Schema.Field;
-import org.apache.avro.Schema.Type;
-import org.apache.avro.generic.GenericData;
+import org.apache.avro.specific.SpecificData;
 import org.apache.avro.specific.SpecificRecord;
-import org.apache.gora.avro.PersistentDatumReader;
-import org.apache.gora.persistency.ListGenericArray;
+import org.apache.avro.specific.SpecificRecordBase;
+import org.apache.gora.persistency.Dirtyable;
 import org.apache.gora.persistency.Persistent;
-import org.apache.gora.persistency.StateManager;
-import org.apache.gora.persistency.StatefulHashMap;
 
 /**
- * Base classs implementing common functionality for Persistent
- * classes.
- */
-public abstract class PersistentBase implements Persistent, SpecificRecord {
-
-  protected static Map<Class<?>, Map<String, Integer>> FIELD_MAP =
-    new HashMap<Class<?>, Map<String,Integer>>();
-
-  protected static Map<Class<?>, String[]> FIELDS =
-    new HashMap<Class<?>, String[]>();
-
-  protected static final PersistentDatumReader<PersistentBase> datumReader =
-    new PersistentDatumReader<PersistentBase>();
-    
-  private StateManager stateManager;
-
-  protected PersistentBase() {
-    this(new StateManagerImpl());
-  }
-
-  protected PersistentBase(StateManager stateManager) {
-    this.stateManager = stateManager;
-    stateManager.setManagedPersistent(this);
-  }
-
-  /** Subclasses should call this function for all the persistable fields
-   * in the class to register them.
-   * @param clazz the Persistent class
-   * @param fields the name of the fields of the class
-   */
-  protected static void registerFields(Class<?> clazz, String... fields) {
-    FIELDS.put(clazz, fields);
-    int fieldsLength = fields == null ? 0 :fields.length;
-    HashMap<String, Integer> map = new HashMap<String, Integer>(fieldsLength);
-
-    for(int i=0; i < fieldsLength; i++) {
-      map.put(fields[i], i);
+* Base classs implementing common functionality for Persistent classes.
+*/
+public abstract class PersistentBase extends SpecificRecordBase implements
+    Persistent {
+
+  public static class PersistentData extends SpecificData {
+    private static final PersistentData INSTANCE = new PersistentData();
+
+    public static PersistentData get() {
+      return INSTANCE;
+    }
+
+    public boolean equals(SpecificRecord obj1, SpecificRecord that) {
+      if (that == obj1)
+        return true; // identical object
+      if (!(that instanceof SpecificRecord))
+        return false; // not a record
+      if (obj1.getClass() != that.getClass())
+        return false; // not same schema
+      return PersistentData.get().compare(obj1, that, obj1.getSchema(), true) == 0;
     }
-    FIELD_MAP.put(clazz, map);
-  }
-
-  @Override
-  public StateManager getStateManager() {
-    return stateManager;
-  }
-
-  @Override
-  public String[] getFields() {
-    return FIELDS.get(getClass());
-  }
 
-  @Override
-  public String getField(int index) {
-    return FIELDS.get(getClass())[index];
   }
 
   @Override
-  public int getFieldIndex(String field) {
-    return FIELD_MAP.get(getClass()).get(field);
+  public void clearDirty() {
+    ByteBuffer dirtyBytes = getDirtyBytes();
+    assert (dirtyBytes.position() == 0);
+    for (int i = 0; i < dirtyBytes.limit(); i++) {
+      dirtyBytes.put(i, (byte) 0);
+    }
+    for (Field field : getSchema().getFields()) {
+      clearDirynessIfFieldIsDirtyable(field.pos());
+    }
   }
 
-  @Override
-  @SuppressWarnings("rawtypes")
-  public void clear() {
-    List<Field> fields = getSchema().getFields();
-
-    for(int i=0; i<getFields().length; i++) {
-      switch(fields.get(i).schema().getType()) {
-        case MAP: 
-          if(get(i) != null) {
-            if (get(i) instanceof StatefulHashMap) {
-              ((StatefulHashMap)get(i)).reuse(); 
-            } else {
-              ((Map)get(i)).clear();
-            }
-          }
-          break;
-        case ARRAY:
-          if(get(i) != null) {
-            if(get(i) instanceof ListGenericArray) {
-              ((ListGenericArray)get(i)).clear();
-            } else {
-              put(i, new ListGenericArray(fields.get(i).schema()));
-            }
-          }
-          break;
-        case RECORD :
-          Persistent field = ((Persistent)get(i));
-          if(field != null) field.clear();
-          break;
-        case BOOLEAN: put(i, false); break;
-        case INT    : put(i, 0); break;
-        case DOUBLE : put(i, 0d); break;
-        case FLOAT  : put(i, 0f); break;
-        case LONG   : put(i, 0l); break;
-        case NULL   : break;
-        default     : put(i, null); break;
-      }
+  private void clearDirynessIfFieldIsDirtyable(int fieldIndex) {
+    if (fieldIndex == 0)
+      return;
+    Object value = get(fieldIndex);
+    if (value instanceof Dirtyable) {
+      ((Dirtyable) value).clearDirty();
     }
-    clearDirty();
-    clearReadable();
   }
 
   @Override
-  public boolean isNew() {
-    return getStateManager().isNew(this);
+  public void clearDirty(int fieldIndex) {
+    ByteBuffer dirtyBytes = getDirtyBytes();
+    assert (dirtyBytes.position() == 0);
+    int byteOffset = fieldIndex / 8;
+    int bitOffset = fieldIndex % 8;
+    byte currentByte = dirtyBytes.get(byteOffset);
+    currentByte = (byte) ((~(1 << bitOffset)) & currentByte);
+    dirtyBytes.put(byteOffset, currentByte);
+    clearDirynessIfFieldIsDirtyable(fieldIndex);
   }
 
   @Override
-  public void setNew() {
-    getStateManager().setNew(this);
+  public void clearDirty(String field) {
+    clearDirty(getSchema().getField(field).pos());
   }
 
   @Override
-  public void clearNew() {
-    getStateManager().clearNew(this);
+  public boolean isDirty() {
+    List<Field> fields = getSchema().getFields();
+    boolean isSubRecordDirty = false;
+    for (Field field : fields) {
+      isSubRecordDirty = isSubRecordDirty || checkIfMutableFieldAndDirty(field);
+    }
+    ByteBuffer dirtyBytes = getDirtyBytes();
+    assert (dirtyBytes.position() == 0);
+    boolean dirty = false;
+    for (int i = 0; i < dirtyBytes.limit(); i++) {
+      dirty = dirty || dirtyBytes.get(i) != 0;
+    }
+    return isSubRecordDirty || dirty;
   }
 
-  @Override
-  public boolean isDirty() {
-    return getStateManager().isDirty(this);
+  private boolean checkIfMutableFieldAndDirty(Field field) {
+    if (field.pos() == 0)
+      return false;
+    switch (field.schema().getType()) {
+    case RECORD:
+    case MAP:
+    case ARRAY:
+      Object value = get(field.pos());
+      return !(value instanceof Dirtyable) || value==null ? false : ((Dirtyable) value).isDirty();
+    case UNION:
+      value = get(field.pos());
+      return !(value instanceof Dirtyable) || value==null ? false : ((Dirtyable) value).isDirty();
+    default:
+      break;
+    }
+    return false;
   }
 
   @Override
   public boolean isDirty(int fieldIndex) {
-    return getStateManager().isDirty(this, fieldIndex);
+    Field field = getSchema().getFields().get(fieldIndex);
+    boolean isSubRecordDirty = checkIfMutableFieldAndDirty(field);
+    ByteBuffer dirtyBytes = getDirtyBytes();
+    assert (dirtyBytes.position() == 0);
+    int byteOffset = fieldIndex / 8;
+    int bitOffset = fieldIndex % 8;
+    byte currentByte = dirtyBytes.get(byteOffset);
+    return isSubRecordDirty || 0 != ((1 << bitOffset) & currentByte);
   }
 
   @Override
-  public boolean isDirty(String field) {
-    return isDirty(getFieldIndex(field));
+  public boolean isDirty(String fieldName) {
+    Field field = getSchema().getField(fieldName);
+    if(field == null){
+      throw new IndexOutOfBoundsException
+      ("Field "+ fieldName + " does not exist in this schema.");
+    }
+    return isDirty(field.pos());
   }
 
   @Override
   public void setDirty() {
-    getStateManager().setDirty(this);
+    ByteBuffer dirtyBytes = getDirtyBytes();
+    assert (dirtyBytes.position() == 0);
+    for (int i = 0; i < dirtyBytes.limit(); i++) {
+      dirtyBytes.put(i, (byte) -128);
+    }
   }
 
   @Override
   public void setDirty(int fieldIndex) {
-    getStateManager().setDirty(this, fieldIndex);
+    ByteBuffer dirtyBytes = getDirtyBytes();
+    assert (dirtyBytes.position() == 0);
+    int byteOffset = fieldIndex / 8;
+    int bitOffset = fieldIndex % 8;
+    byte currentByte = dirtyBytes.get(byteOffset);
+    currentByte = (byte) ((1 << bitOffset) | currentByte);
+    dirtyBytes.put(byteOffset, currentByte);
   }
 
   @Override
   public void setDirty(String field) {
-    setDirty(getFieldIndex(field));
+    setDirty(getSchema().getField(field).pos());
   }
 
-  @Override
-  public void clearDirty(int fieldIndex) {
-    getStateManager().clearDirty(this, fieldIndex);
+  private ByteBuffer getDirtyBytes() {
+    return (ByteBuffer) get(0);
   }
 
   @Override
-  public void clearDirty(String field) {
-    clearDirty(getFieldIndex(field));
-  }
-
-  @Override
-  public void clearDirty() {
-    getStateManager().clearDirty(this);
-  }
-
-  @Override
-  public boolean isReadable(int fieldIndex) {
-    return getStateManager().isReadable(this, fieldIndex);
-  }
-
-  @Override
-  public boolean isReadable(String field) {
-    return isReadable(getFieldIndex(field));
-  }
-
-  @Override
-  public void setReadable(int fieldIndex) {
-    getStateManager().setReadable(this, fieldIndex);
-  }
-
-  @Override
-  public void setReadable(String field) {
-    setReadable(getFieldIndex(field));
-  }
-
-  @Override
-  public void clearReadable() {
-    getStateManager().clearReadable(this);
-  }
-
-  @Override
-  public void clearReadable(int fieldIndex) {
-    getStateManager().clearReadable(this, fieldIndex);
-  }
-
-  @Override
-  public void clearReadable(String field) {
-    clearReadable(getFieldIndex(field));
-  }
-
-  //@Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (!(o instanceof SpecificRecord)) return false;
-
-    SpecificRecord r2 = (SpecificRecord)o;
-    if (!this.getSchema().equals(r2.getSchema())) return false;
-
-    return this.hashCode() == r2.hashCode();
-  }
-
-  //@Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    List<Field> fields = this.getSchema().getFields();
-    int end = fields.size();
-    for (int i = 0; i < end; i++) {
-      result = prime * result + getFieldHashCode(i, fields.get(i));
-    }
-    return result;
-  }
-
-  /**
-   * Computes a (record's) field's hash code.
-   * @param i Index of the field in the actual
-   * @param field
-   * @return
-   */
-  private int getFieldHashCode(int i, Field field) {
-    Object o = get(i);
-    if(o == null)
-      return 0;
-
-    // XXX Union special case: in a field being union we have to check the
-    // inner schemas for Type.BYTES special case, but because it is not a
-    // field we check it this way. Too simple case to create another
-    // private method
-    boolean isUnionField = false ;
-    int unionIndex = -1 ;
-    
-    if (field.schema().getType() == Type.UNION) {
-      isUnionField = true ;
-      unionIndex = GenericData.get().resolveUnion(field.schema(), o);
-    }
-    
-    if(field.schema().getType() == Type.BYTES
-       || (isUnionField
-           && field.schema().getTypes().get(unionIndex).getType() == Type.BYTES)) {
-      // ByteBuffer.hashCode() depends on internal 'position' index, but we must ignore that.
-      return getByteBufferHashCode((ByteBuffer)o);
+  public void clear() {
+    Collection<Field> unmanagedFields = getUnmanagedFields();
+    for (Field field : getSchema().getFields()) {
+      if (!unmanagedFields.contains(field))
+        continue;
+      put(field.pos(), PersistentData.get().deepCopy(field.schema(), PersistentData.get().getDefaultValue(field)));
     }
-
-    return o.hashCode();
+    clearDirty();
   }
 
-  /** ByteBuffer.hashCode() takes into account the position of the
-   * buffer, but we do not want that*/
-  private int getByteBufferHashCode(ByteBuffer buf) {
-    int h = 1;
-    int p = buf.arrayOffset();
-    for (int j = buf.limit() - 1; j >= p; j--)
-          h = 31 * h + buf.get(j);
-    return h;
-  }
-  
   @Override
-  public Persistent clone() {
-    return datumReader.clone(this, getSchema());
+  public boolean equals(Object that) {
+    if (that == this) {
+      return true;
+    } else if (that instanceof Persistent) {
+      return PersistentData.get().equals(this, (SpecificRecord) that);
+    } else {
+      return false;
+    }
   }
   
-  //@Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    builder.append(super.toString());
-    builder.append(" {\n");
+  public List<Field> getUnmanagedFields(){
     List<Field> fields = getSchema().getFields();
-    for(int i=0; i<fields.size(); i++) {
-      builder.append("  \"").append(fields.get(i).name()).append("\":\"");
-      builder.append(get(i)).append("\"\n");
-    }
-    builder.append("}");
-    return builder.toString();
+    return fields.subList(1, fields.size());
   }
   
-  protected boolean isFieldEqual(int index, Object value) {
-    Object old = get(index);
-    if (old == null && value == null)
-      return true;
-    if (old == null || value == null)
-      return false;
-    return value.equals(old);
-  }
 }

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/StateManagerImpl.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/StateManagerImpl.java?rev=1586888&r1=1586887&r2=1586888&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/StateManagerImpl.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/impl/StateManagerImpl.java Sat Apr 12 19:21:53 2014
@@ -1,104 +0,0 @@
-/**
- * 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.persistency.impl;
-
-import java.util.BitSet;
-
-import org.apache.gora.persistency.Persistent;
-import org.apache.gora.persistency.StateManager;
-
-/**
- * An implementation for the StateManager. This implementation assumes 
- * every Persistent object has it's own StateManager.
- */
-public class StateManagerImpl implements StateManager {
-
-  //TODO: serialize isNew in PersistentSerializer 
-  protected boolean isNew;
-  protected BitSet dirtyBits;
-  protected BitSet readableBits;
-
-  public StateManagerImpl() {
-  }
-
-  public void setManagedPersistent(Persistent persistent) {
-    dirtyBits = new BitSet(((PersistentBase)persistent).getSchema().getFields().size());
-    readableBits = new BitSet(((PersistentBase)persistent).getSchema().getFields().size());
-    isNew = true;
-  }
-
-  @Override
-  public boolean isNew(Persistent persistent) {
-    return isNew;
-  }
-  
-  @Override
-  public void setNew(Persistent persistent) {
-    this.isNew = true;
-  }
-  
-  @Override
-  public void clearNew(Persistent persistent) {
-    this.isNew = false;
-  }
-  
-  public void setDirty(Persistent persistent, int fieldIndex) {
-    dirtyBits.set(fieldIndex);
-    readableBits.set(fieldIndex);
-  }
-  
-  public boolean isDirty(Persistent persistent, int fieldIndex) {
-    return dirtyBits.get(fieldIndex);
-  }
-
-  public boolean isDirty(Persistent persistent) {
-    return !dirtyBits.isEmpty();
-  }
-  
-  @Override
-  public void setDirty(Persistent persistent) {
-    dirtyBits.set(0, dirtyBits.size());
-  }
-  
-  @Override
-  public void clearDirty(Persistent persistent, int fieldIndex) {
-    dirtyBits.clear(fieldIndex);
-  }
-  
-  public void clearDirty(Persistent persistent) {
-    dirtyBits.clear();
-  }
-  
-  public void setReadable(Persistent persistent, int fieldIndex) {
-    readableBits.set(fieldIndex);
-  }
-
-  public boolean isReadable(Persistent persistent, int fieldIndex) {
-    return readableBits.get(fieldIndex);
-  }
-
-  @Override
-  public void clearReadable(Persistent persistent, int fieldIndex) {
-    readableBits.clear(fieldIndex);
-  }
-  
-  public void clearReadable(Persistent persistent) {
-    readableBits.clear();
-  }
-}

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/ws/impl/BeanFactoryWSImpl.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/ws/impl/BeanFactoryWSImpl.java?rev=1586888&r1=1586887&r2=1586888&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/ws/impl/BeanFactoryWSImpl.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/ws/impl/BeanFactoryWSImpl.java Sat Apr 12 19:21:53 2014
@@ -91,7 +91,7 @@ public class BeanFactoryWSImpl<K, T exte
   public K newKey() throws Exception {
     // TODO this method should be checked to see how object states will be managed
     if(isKeyPersistent)
-      return (K)((Persistent)key).newInstance(new StateManagerWSImpl());
+      return keyClass.newInstance();
     else if(keyConstructor == null) {
       throw new RuntimeException("Key class does not have a no-arg constructor");
     }
@@ -99,13 +99,19 @@ public class BeanFactoryWSImpl<K, T exte
       return keyConstructor.newInstance(ReflectionUtils.EMPTY_OBJECT_ARRAY);
   }
  
-  @SuppressWarnings("unchecked")
   @Override
   /**
    * Creates a new persistent object
    */
   public T newPersistent() {
-    return (T) persistent.newInstance(new StateManagerWSImpl());
+    try {
+      return (T) persistentClass.newInstance();
+    } catch (InstantiationException e) {
+      throw new RuntimeException(e);
+    } catch (IllegalAccessException e) {
+      e.printStackTrace();
+      throw new RuntimeException(e);
+    }
   }
   
   @Override

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/ws/impl/PersistentWSBase.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/ws/impl/PersistentWSBase.java?rev=1586888&r1=1586887&r2=1586888&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/ws/impl/PersistentWSBase.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/ws/impl/PersistentWSBase.java Sat Apr 12 19:21:53 2014
@@ -23,7 +23,6 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.gora.persistency.Persistent;
-import org.apache.gora.persistency.StateManager;
 
 /**
  * Base classs implementing common functionality for Web services 
@@ -43,26 +42,6 @@ public abstract class PersistentWSBase i
   protected static Map<Class<?>, String[]> FIELDS =
     new HashMap<Class<?>, String[]>();
     
-  /**
-   * Object used to manage the state of fields
-   */
-  private StateManager stateManager;
-
-  /**
-   * Constructor
-   */
-  protected PersistentWSBase() {
-    this(new StateManagerWSImpl());
-  }
-
-  /**
-   * Constructor using a stateManager object
-   * @param stateManager
-   */
-  protected PersistentWSBase(StateManager stateManager) {
-    this.stateManager = stateManager;
-    stateManager.setManagedPersistent(this);
-  }
 
   /** Subclasses should call this function for all the persistable fields
    * in the class to register them.
@@ -80,15 +59,6 @@ public abstract class PersistentWSBase i
     FIELD_MAP.put(clazz, map);
   }
 
-  @Override
-  /**
-   * Gets the state manager
-   */
-  public StateManager getStateManager() {
-    return stateManager;
-  }
-
-  @Override
   /**
    * Gets fields using a specific class
    */
@@ -96,7 +66,6 @@ public abstract class PersistentWSBase i
     return FIELDS.get(getClass());
   }
 
-  @Override
   /**
    * Gets a specific field from the fields map
    */
@@ -104,7 +73,6 @@ public abstract class PersistentWSBase i
     return FIELDS.get(getClass())[index];
   }
 
-  @Override
   /**
    * Gets a field index based on the field name
    */
@@ -122,36 +90,17 @@ public abstract class PersistentWSBase i
     clearReadable();
   }
 
-  @Override
-  /**
-   * Determines if a class is new or not
-   */
-  public boolean isNew() {
-    return getStateManager().isNew(this);
-  }
-
-  @Override
-  /**
-   * Sets this element as a new one inside the stateManager object
-   */
-  public void setNew() {
-    getStateManager().setNew(this);
-  }
-
-  @Override
-  /**
-   * Clears a new object from the stateManager
-   */
-  public void clearNew() {
-    getStateManager().clearNew(this);
+  private void clearReadable() {
+	  // TODO Auto-generated method stub
+	  
   }
 
-  @Override
+	@Override
   /**
    * Determines if an object has been modified or not
    */
   public boolean isDirty() {
-    return getStateManager().isDirty(this);
+    return true;
   }
 
   @Override
@@ -160,7 +109,7 @@ public abstract class PersistentWSBase i
    * based on its field index
    */
   public boolean isDirty(int fieldIndex) {
-    return getStateManager().isDirty(this, fieldIndex);
+    return true;
   }
 
   @Override
@@ -174,112 +123,6 @@ public abstract class PersistentWSBase i
 
   @Override
   /**
-   * Sets this class as dirty
-   */
-  public void setDirty() {
-    getStateManager().setDirty(this);
-  }
-
-  @Override
-  /**
-   * Sets a specific field as dirty using its index
-   */
-  public void setDirty(int fieldIndex) {
-    getStateManager().setDirty(this, fieldIndex);
-  }
-
-  @Override
-  /**
-   * Sets a specific field as dirty using its name
-   */
-  public void setDirty(String field) {
-    setDirty(getFieldIndex(field));
-  }
-
-  @Override
-  /**
-   * Clears dirty fields using its index
-   */
-  public void clearDirty(int fieldIndex) {
-    getStateManager().clearDirty(this, fieldIndex);
-  }
-
-  @Override
-  /**
-   * Clears dirty fields using its name
-   */
-  public void clearDirty(String field) {
-    clearDirty(getFieldIndex(field));
-  }
-
-  @Override
-  /**
-   * Clears dirty fields from the state manager
-   */
-  public void clearDirty() {
-    getStateManager().clearDirty(this);
-  }
-
-  @Override
-  /**
-   * Checks if a field is readable using its index
-   */
-  public boolean isReadable(int fieldIndex) {
-    return getStateManager().isReadable(this, fieldIndex);
-  }
-
-  @Override
-  /**
-   * Checks if a field is readable using its name
-   */
-  public boolean isReadable(String field) {
-    return isReadable(getFieldIndex(field));
-  }
-
-  @Override
-  /**
-   * Sets a field as readable using its index
-   */
-  public void setReadable(int fieldIndex) {
-    getStateManager().setReadable(this, fieldIndex);
-  }
-
-  @Override
-  /**
-   * Sets a field as readable using its name
-   */
-  public void setReadable(String field) {
-    setReadable(getFieldIndex(field));
-  }
-
-  @Override
-  /**
-   * Clears this readable object from the state manager
-   */
-  public void clearReadable() {
-    getStateManager().clearReadable(this);
-  }
-
-  @Override
-  /**
-   * Clears a readable object based on its field index
-   * using a stateManager object
-   */
-  public void clearReadable(int fieldIndex) {
-    getStateManager().clearReadable(this, fieldIndex);
-  }
-
-  @Override
-  /**
-   * Clears a readable object based on its field name
-   * using a stateManager object
-   */
-  public void clearReadable(String field) {
-    clearReadable(getFieldIndex(field));
-  }
-
-  @Override
-  /**
    * Determines if an object is equal to this class
    */
   public boolean equals(Object o) {

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/ws/impl/StateManagerWSImpl.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/ws/impl/StateManagerWSImpl.java?rev=1586888&r1=1586887&r2=1586888&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/ws/impl/StateManagerWSImpl.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/persistency/ws/impl/StateManagerWSImpl.java Sat Apr 12 19:21:53 2014
@@ -1,149 +0,0 @@
-/**
- * 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.persistency.ws.impl;
-
-import java.util.BitSet;
-
-import org.apache.gora.persistency.Persistent;
-import org.apache.gora.persistency.StateManager;
-
-/**
- * An implementation for the StateManager. This implementation assumes 
- * every Persistent object has it's own StateManager.
- */
-public class StateManagerWSImpl implements StateManager {
-
-  //TODO: serialize isNew in PersistentSerializer 
-  protected boolean isNew;
-  protected BitSet dirtyBits;
-  protected BitSet readableBits;
-
-  /**
-   * Constructor
-   */
-  public StateManagerWSImpl() {
-  }
-
-  /**
-   * Sets dirtyBits and readableBits sizes
-   */
-  public void setManagedPersistent(Persistent persistent) {
-   // dirtyBits = new BitSet(persistent.getSchema().getFields().size());
-   // readableBits = new BitSet(persistent.getSchema().getFields().size());
-    isNew = true;
-  }
-
-  @Override
-  /**
-   * Checks if an object is new or not
-   */
-  public boolean isNew(Persistent persistent) {
-    return isNew;
-  }
-  
-  @Override
-  /**
-   * Sets an object as new
-   */
-  public void setNew(Persistent persistent) {
-    this.isNew = true;
-  }
-  
-  @Override
-  /**
-   * Clear the new object by setting it as not new
-   */
-  public void clearNew(Persistent persistent) {
-    this.isNew = false;
-  }
-  
-  /**
-   * Sets an object as dirty using its index
-   */
-  public void setDirty(Persistent persistent, int fieldIndex) {
-    dirtyBits.set(fieldIndex);
-    readableBits.set(fieldIndex);
-  }
-  
-  /**
-   * Determines if an object is dirty or not based on its index
-   */
-  public boolean isDirty(Persistent persistent, int fieldIndex) {
-    return dirtyBits.get(fieldIndex);
-  }
-
-  /**
-   * Determines if an object is dirty
-   */
-  public boolean isDirty(Persistent persistent) {
-    return !dirtyBits.isEmpty();
-  }
-  
-  @Override
-  /**
-   * Sets an object as dirty
-   */
-  public void setDirty(Persistent persistent) {
-    dirtyBits.set(0, dirtyBits.size());
-  }
-  
-  @Override
-  /**
-   * Marks a persistent object as not dirty using its index
-   */
-  public void clearDirty(Persistent persistent, int fieldIndex) {
-    dirtyBits.clear(fieldIndex);
-  }
-  
-  /**
-   * Marks all objects as not dirty
-   */
-  public void clearDirty(Persistent persistent) {
-    dirtyBits.clear();
-  }
-  
-  /**
-   * Sets a persistent object as readable using its index
-   */
-  public void setReadable(Persistent persistent, int fieldIndex) {
-    readableBits.set(fieldIndex);
-  }
-
-  /**
-   * Determines if an object is readable using its index
-   */
-  public boolean isReadable(Persistent persistent, int fieldIndex) {
-    return readableBits.get(fieldIndex);
-  }
-
-  @Override
-  /**
-   * Marks an object as non-readable using its index
-   */
-  public void clearReadable(Persistent persistent, int fieldIndex) {
-    readableBits.clear(fieldIndex);
-  }
-  
-  /**
-   * Marks all objects as non-readable
-   */
-  public void clearReadable(Persistent persistent) {
-    readableBits.clear();
-  }
-}

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/store/DataStoreFactory.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/store/DataStoreFactory.java?rev=1586888&r1=1586887&r2=1586888&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/store/DataStoreFactory.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/store/DataStoreFactory.java Sat Apr 12 19:21:53 2014
@@ -56,7 +56,7 @@ public class DataStoreFactory{
 
   public static final String MAPPING_FILE = "mapping.file";
 
-	public static final String SCHEMA_NAME = "schema.name";
+  public static final String SCHEMA_NAME = "schema.name";
 
   /**
    * Do not use! Deprecated because it shares system wide state. 
@@ -64,7 +64,7 @@ public class DataStoreFactory{
    */
   @Deprecated()
   public static final Properties properties = createProps();
-  
+
   /**
    * Creates a new {@link Properties}. It adds the default gora configuration
    * resources. This properties object can be modified and used to instantiate
@@ -75,9 +75,9 @@ public class DataStoreFactory{
    */
   public static Properties createProps() {
     try {
-    Properties properties = new Properties();
+      Properties properties = new Properties();
       InputStream stream = DataStoreFactory.class.getClassLoader()
-        .getResourceAsStream(GORA_DEFAULT_PROPERTIES_FILE);
+          .getResourceAsStream(GORA_DEFAULT_PROPERTIES_FILE);
       if(stream != null) {
         try {
           properties.load(stream);
@@ -150,11 +150,11 @@ public class DataStoreFactory{
   public static <D extends DataStore<K,T>, K, T extends Persistent>
   D createDataStore(Class<D> dataStoreClass, Class<K> keyClass
       , Class<T> persistent, Configuration conf, Properties properties, String schemaName) 
-  throws GoraException {
+          throws GoraException {
     try {
       setDefaultSchemaName(properties, schemaName);
       D dataStore =
-        ReflectionUtils.newInstance(dataStoreClass);
+          ReflectionUtils.newInstance(dataStoreClass);
       if ((dataStore instanceof Configurable) && conf != null) {
         ((Configurable)dataStore).setConf(conf);
       }
@@ -182,7 +182,7 @@ public class DataStoreFactory{
   public static <D extends DataStore<K,T>, K, T extends Persistent>
   D createDataStore(Class<D> dataStoreClass
       , Class<K> keyClass, Class<T> persistent, Configuration conf, Properties properties) 
-  throws GoraException {
+          throws GoraException {
     return createDataStore(dataStoreClass, keyClass, persistent, conf, properties, null);
   }
 
@@ -216,10 +216,10 @@ public class DataStoreFactory{
   @SuppressWarnings("unchecked")
   public static <K, T extends Persistent> DataStore<K, T> getDataStore(
       String dataStoreClass, Class<K> keyClass, Class<T> persistentClass, Configuration conf)
-      throws GoraException {
+          throws GoraException {
     try {
       Class<? extends DataStore<K,T>> c
-        = (Class<? extends DataStore<K, T>>) Class.forName(dataStoreClass);
+          = (Class<? extends DataStore<K, T>>) Class.forName(dataStoreClass);
       return createDataStore(c, keyClass, persistentClass, conf, createProps(), null);
     } catch(GoraException ex) {
       throw ex;
@@ -241,7 +241,7 @@ public class DataStoreFactory{
   @SuppressWarnings({ "unchecked" })
   public static <K, T extends Persistent> DataStore<K, T> getDataStore(
       String dataStoreClass, String keyClass, String persistentClass, Configuration conf)
-    throws GoraException {
+          throws GoraException {
 
     try {
       Class<? extends DataStore<K,T>> c
@@ -362,6 +362,17 @@ public class DataStoreFactory{
     return findProperty(properties, store, OUTPUT_PATH, null);
   }
 
+  /**
+   * Looks for the <code>gora-&lt;classname&gt;-mapping.xml</code> as a resource 
+   * on the classpath. This can however also be specified within the 
+   * <code>gora.properties</code> file with the key 
+   * <code>gora.&lt;classname&gt;.mapping.file=</code>.
+   * @param properties which hold keys from which we can obtain values for datastore mappings.
+   * @param store {@link org.apache.gora.store.DataStore} object to get the mapping for.
+   * @param defaultValue default value for the <code>gora-&lt;classname&gt;-mapping.xml</code>
+   * @return mappingFilename if one is located.
+   * @throws IOException if there is a problem reading or obtaining the mapping file.
+   */
   public static String getMappingFile(Properties properties, DataStore<?,?> store
       , String defaultValue) throws IOException {