You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gora.apache.org by fe...@apache.org on 2012/03/29 15:45:04 UTC

svn commit: r1306866 - in /gora/trunk: ./ gora-core/src/main/java/org/apache/gora/store/ gora-core/src/main/java/org/apache/gora/store/impl/ gora-core/src/main/java/org/apache/gora/util/ gora-core/src/test/java/org/apache/gora/ gora-core/src/test/java/...

Author: ferdy
Date: Thu Mar 29 13:45:03 2012
New Revision: 1306866

URL: http://svn.apache.org/viewvc?rev=1306866&view=rev
Log:
GORA-105 DataStoreFactory does not properly support multiple stores

Added:
    gora/trunk/gora-core/src/main/java/org/apache/gora/util/WritableUtils.java   (with props)
    gora/trunk/gora-core/src/test/java/org/apache/gora/util/TestWritableUtils.java   (with props)
Modified:
    gora/trunk/CHANGES.txt
    gora/trunk/gora-core/src/main/java/org/apache/gora/store/DataStoreFactory.java
    gora/trunk/gora-core/src/main/java/org/apache/gora/store/impl/DataStoreBase.java
    gora/trunk/gora-core/src/test/java/org/apache/gora/GoraTestDriver.java
    gora/trunk/gora-core/src/test/java/org/apache/gora/avro/mapreduce/TestDataFileAvroStoreMapReduce.java
    gora/trunk/gora-core/src/test/java/org/apache/gora/avro/store/TestAvroStore.java
    gora/trunk/gora-core/src/test/java/org/apache/gora/store/TestDataStoreFactory.java

Modified: gora/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/gora/trunk/CHANGES.txt?rev=1306866&r1=1306865&r2=1306866&view=diff
==============================================================================
--- gora/trunk/CHANGES.txt (original)
+++ gora/trunk/CHANGES.txt Thu Mar 29 13:45:03 2012
@@ -5,6 +5,7 @@
 Gora Change Log
 
 Trunk (unreleased changes):
+* GORA-105 DataStoreFactory does not properly support multiple stores (ferdy)
 
 * GORA-** Update Gora parent pom to include maven release plugin targets, and update developer credentials.
 

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=1306866&r1=1306865&r2=1306866&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 Thu Mar 29 13:45:03 2012
@@ -19,7 +19,6 @@ package org.apache.gora.store;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.HashMap;
 import java.util.Properties;
 
 import org.apache.commons.logging.Log;
@@ -57,22 +56,41 @@ 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";
 
-  private static String propertiesFile = GORA_DEFAULT_PROPERTIES_FILE;
-
-  private static String defaultDataStoreClass;
-
-  private static HashMap<Integer, DataStore<?,?>> dataStores;
-
-  public static Properties properties;
-
-  static {
-    dataStores = new HashMap<Integer, DataStore<?,?>>();
+  /**
+   * Do not use! Deprecated because it shares system wide state. 
+   * Use {@link #createProps()} instead.
+   */
+  @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
+   * store instances. It is recommended to use a properties object for a single
+   * store, because the properties object is passed on to store initialization
+   * methods that are able to store the properties as a field.   
+   * @return The new properties object.
+   */
+  public static Properties createProps() {
     try {
-      readProperties();
-    } catch (IOException ex) {
-      throw new RuntimeException(ex);
+    Properties properties = new Properties();
+      InputStream stream = DataStoreFactory.class.getClassLoader()
+        .getResourceAsStream(GORA_DEFAULT_PROPERTIES_FILE);
+      if(stream != null) {
+        try {
+          properties.load(stream);
+          return properties;
+        } finally {
+          stream.close();
+        }
+      } else {
+        log.warn(GORA_DEFAULT_PROPERTIES_FILE + " not found, properties will be empty.");
+      }
+      return properties;
+    } catch(Exception e) {
+      throw new RuntimeException(e);
     }
   }
 
@@ -84,18 +102,51 @@ public class DataStoreFactory {
     dataStore.initialize(keyClass, persistent, properties);
   }
 
+  /**
+   * Instantiate a new {@link DataStore}. Uses default properties. Uses 'null' schema.
+   * 
+   * @param dataStoreClass The datastore implementation class.
+   * @param keyClass The key class.
+   * @param persistent The value class.
+   * @param conf {@link Configuration} to be used be the store.
+   * @return A new store instance.
+   * @throws GoraException
+   */
   public static <D extends DataStore<K,T>, K, T extends Persistent>
   D createDataStore(Class<D> dataStoreClass
       , Class<K> keyClass, Class<T> persistent, Configuration conf) throws GoraException {
-    return createDataStore(dataStoreClass, keyClass, persistent, conf, properties);
+    return createDataStore(dataStoreClass, keyClass, persistent, conf, createProps(), null);
   }
 
+  /**
+   * Instantiate a new {@link DataStore}. Uses default properties.
+   * 
+   * @param dataStoreClass The datastore implementation class.
+   * @param keyClass The key class.
+   * @param persistent The value class.
+   * @param conf {@link Configuration} to be used be the store.
+   * @param schemaName A default schemaname that will be put on the properties.
+   * @return A new store instance.
+   * @throws GoraException
+   */
   public static <D extends DataStore<K,T>, K, T extends Persistent>
   D createDataStore(Class<D> dataStoreClass , Class<K> keyClass, 
       Class<T> persistent, Configuration conf, String schemaName) throws GoraException {
-    return createDataStore(dataStoreClass, keyClass, persistent, conf, properties, schemaName);
+    return createDataStore(dataStoreClass, keyClass, persistent, conf, createProps(), schemaName);
   }
 
+  /**
+   * Instantiate a new {@link DataStore}.
+   * 
+   * @param dataStoreClass The datastore implementation class.
+   * @param keyClass The key class.
+   * @param persistent The value class.
+   * @param conf {@link Configuration} to be used be the store.
+   * @param properties The properties to be used be the store.
+   * @param schemaName A default schemaname that will be put on the properties.
+   * @return A new store instance.
+   * @throws GoraException
+   */
   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) 
@@ -117,6 +168,17 @@ public class DataStoreFactory {
     }
   }
 
+  /**
+   * Instantiate a new {@link DataStore}. Uses 'null' schema.
+   * 
+   * @param dataStoreClass The datastore implementation class.
+   * @param keyClass The key class.
+   * @param persistent The value class.
+   * @param conf {@link Configuration} to be used be the store.
+   * @param properties The properties to be used be the store.
+   * @return A new store instance.
+   * @throws GoraException
+   */
   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) 
@@ -124,29 +186,41 @@ public class DataStoreFactory {
     return createDataStore(dataStoreClass, keyClass, persistent, conf, properties, null);
   }
 
-  @SuppressWarnings("unchecked")
+  /**
+   * Instantiate a new {@link DataStore}. Uses default properties. Uses 'null' schema.
+   * 
+   * @param dataStoreClass The datastore implementation class.
+   * @param keyClass The key class.
+   * @param persistent The value class.
+   * @param conf {@link Configuration} to be used be the store.
+   * @return A new store instance.
+   * @throws GoraException
+   */
   public static <D extends DataStore<K,T>, K, T extends Persistent>
   D getDataStore( Class<D> dataStoreClass, Class<K> keyClass,
       Class<T> persistentClass, Configuration conf) throws GoraException {
-    int hash = getDataStoreKey(dataStoreClass, keyClass, persistentClass);
 
-    D dataStore = (D) dataStores.get(hash);
-    if(dataStore == null) {
-      dataStore = createDataStore(dataStoreClass, keyClass, persistentClass,
-          conf, properties);
-      dataStores.put(hash, dataStore);
-    }
-    return dataStore;
+    return createDataStore(dataStoreClass, keyClass, persistentClass, conf, createProps(), null);
   }
 
+  /**
+   * Instantiate a new {@link DataStore}. Uses default properties. Uses 'null' schema.
+   * 
+   * @param dataStoreClass The datastore implementation class <i>as string</i>.
+   * @param keyClass The key class.
+   * @param persistent The value class.
+   * @param conf {@link Configuration} to be used be the store.
+   * @return A new store instance.
+   * @throws GoraException
+   */
   @SuppressWarnings("unchecked")
-  public static synchronized <K, T extends Persistent> DataStore<K, T> getDataStore(
+  public static <K, T extends Persistent> DataStore<K, T> getDataStore(
       String dataStoreClass, Class<K> keyClass, Class<T> persistentClass, Configuration conf)
       throws GoraException {
     try {
       Class<? extends DataStore<K,T>> c
         = (Class<? extends DataStore<K, T>>) Class.forName(dataStoreClass);
-      return getDataStore(c, keyClass, persistentClass, conf);
+      return createDataStore(c, keyClass, persistentClass, conf, createProps(), null);
     } catch(GoraException ex) {
       throw ex;
     } catch (Exception ex) {
@@ -154,15 +228,27 @@ public class DataStoreFactory {
     }
   }
 
-  @SuppressWarnings({ "unchecked", "rawtypes" })
-  public static synchronized DataStore getDataStore(
+  /**
+   * Instantiate a new {@link DataStore}. Uses default properties. Uses 'null' schema.
+   * 
+   * @param dataStoreClass The datastore implementation class <i>as string</i>.
+   * @param keyClass The key class <i>as string</i>.
+   * @param persistent The value class <i>as string</i>.
+   * @param conf {@link Configuration} to be used be the store.
+   * @return A new store instance.
+   * @throws GoraException
+   */
+  @SuppressWarnings({ "unchecked" })
+  public static <K, T extends Persistent> DataStore<K, T> getDataStore(
       String dataStoreClass, String keyClass, String persistentClass, Configuration conf)
     throws GoraException {
 
     try {
-      Class k = ClassLoadingUtils.loadClass(keyClass);
-      Class p = ClassLoadingUtils.loadClass(persistentClass);
-      return getDataStore(dataStoreClass, k, p, conf);
+      Class<? extends DataStore<K,T>> c
+          = (Class<? extends DataStore<K, T>>) Class.forName(dataStoreClass);
+      Class<K> k = (Class<K>) ClassLoadingUtils.loadClass(keyClass);
+      Class<T> p = (Class<T>) ClassLoadingUtils.loadClass(persistentClass);
+      return createDataStore(c, k, p, conf, createProps(), null);
     } catch(GoraException ex) {
       throw ex;
     } catch (Exception ex) {
@@ -170,37 +256,26 @@ public class DataStoreFactory {
     }
   }
 
+  /**
+   * Instantiate <i>the default</i> {@link DataStore}. Uses default properties. Uses 'null' schema.
+   * 
+   * @param keyClass The key class.
+   * @param persistent The value class.
+   * @param conf {@link Configuration} to be used be the store.
+   * @return A new store instance.
+   * @throws GoraException
+   */
+  @SuppressWarnings("unchecked")
   public static <K, T extends Persistent> DataStore<K, T> getDataStore(
       Class<K> keyClass, Class<T> persistent, Configuration conf) throws GoraException {
-    return getDataStore(defaultDataStoreClass, keyClass, persistent, conf);
-  }
-
-  private static int getDataStoreKey(
-      Class<?> dataStoreClass, Class<?> keyClass, Class<?> persistent) {
-
-    long hash = (((dataStoreClass.hashCode() * 27L)
-        + keyClass.hashCode()) * 31) + persistent.hashCode();
-
-    return (int)hash;
-  }
-
-  private static Properties readProperties() throws IOException {
-    Properties properties = new Properties();
-    if(propertiesFile != null) {
-      InputStream stream = DataStoreFactory.class.getClassLoader()
-        .getResourceAsStream(propertiesFile);
-      if(stream != null) {
-        try {
-          properties.load(stream);
-          setProperties(properties);
-          return properties;
-        } finally {
-          stream.close();
-        }
-      }
+    Properties createProps = createProps();
+    Class<? extends DataStore<K, T>> c;
+    try {
+      c = (Class<? extends DataStore<K, T>>) Class.forName(getDefaultDataStore(createProps));
+    } catch (Exception ex) {
+      throw new GoraException(ex);
     }
-    log.warn("Gora properties are not loaded!");
-    return null;
+    return createDataStore(c, keyClass, persistent, conf, createProps, null);
   }
 
   /**
@@ -292,9 +367,8 @@ public class DataStoreFactory {
     return findProperty(properties, store, MAPPING_FILE, defaultValue);
   }
 
-  private static void setProperties(Properties properties) {
-    defaultDataStoreClass = getProperty(properties, GORA_DEFAULT_DATASTORE_KEY);
-    DataStoreFactory.properties = properties;
+  private static String getDefaultDataStore(Properties properties) {
+    return getProperty(properties, GORA_DEFAULT_DATASTORE_KEY);
   }
 
   private static String getProperty(Properties properties, String key) {
@@ -313,37 +387,40 @@ public class DataStoreFactory {
   }
 
   /**
-   * Sets a property for all the datastores
+   * Set a property
    */
   private static void setProperty(Properties properties, String baseKey, String value) {
-    if(value != null)
+    if(value != null) {
       properties.setProperty(GORA_DATASTORE + baseKey, value);
+    }
   }
 
   /**
-   * Sets a property for the datastores of the given class
+   * Sets a property for the datastore of the given class
    */
   private static<D extends DataStore<K,T>, K, T extends Persistent>
-    void setProperty(Properties properties, Class<D> dataStoreClass, String baseKey, String value) {
+  void setProperty(Properties properties, Class<D> dataStoreClass, String baseKey, String value) {
     properties.setProperty(GORA+"."+org.apache.gora.util.StringUtils.getClassname(dataStoreClass)+"."+baseKey, value);
   }
 
   /**
-   * Gets the default schema name to be used by the datastore
+   * Gets the default schema name of a given store class 
    */
   public static String getDefaultSchemaName(Properties properties, DataStore<?,?> store) {
     return findProperty(properties, store, SCHEMA_NAME, null);
   }
 
   /**
-   * Sets the default schema name to be used by the datastores
+   * Sets the default schema name.
    */
   public static void setDefaultSchemaName(Properties properties, String schemaName) {
-    setProperty(properties, SCHEMA_NAME, schemaName);
+    if (schemaName != null) {
+      setProperty(properties, SCHEMA_NAME, schemaName);
+    }
   }
 
   /**
-   * Sets the default schema name to be used by the datastores of the given class
+   * Sets the default schema name to be used by the datastore of the given class
    */
   public static<D extends DataStore<K,T>, K, T extends Persistent>
   void setDefaultSchemaName(Properties properties, Class<D> dataStoreClass, String schemaName) {

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/store/impl/DataStoreBase.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/store/impl/DataStoreBase.java?rev=1306866&r1=1306865&r2=1306866&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/store/impl/DataStoreBase.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/store/impl/DataStoreBase.java Thu Mar 29 13:45:03 2012
@@ -22,6 +22,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Properties;
 
 import org.apache.avro.Schema;
@@ -37,8 +38,11 @@ import org.apache.gora.store.DataStoreFa
 import org.apache.gora.util.AvroUtils;
 import org.apache.gora.util.ClassLoadingUtils;
 import org.apache.gora.util.StringUtils;
+import org.apache.gora.util.WritableUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.MapWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
 
 /**
  * A Base class for {@link DataStore}s.
@@ -176,8 +180,8 @@ implements DataStore<K, T> {
     try {
       Class<K> keyClass = (Class<K>) ClassLoadingUtils.loadClass(Text.readString(in));
       Class<T> persistentClass = (Class<T>)ClassLoadingUtils.loadClass(Text.readString(in));
-      initialize(keyClass, persistentClass, DataStoreFactory.properties);
-
+      Properties props = WritableUtils.readProperties(in);
+      initialize(keyClass, persistentClass, props);
     } catch (ClassNotFoundException ex) {
       throw new IOException(ex);
     }
@@ -187,6 +191,7 @@ implements DataStore<K, T> {
   public void write(DataOutput out) throws IOException {
     Text.writeString(out, getKeyClass().getCanonicalName());
     Text.writeString(out, getPersistentClass().getCanonicalName());
+    WritableUtils.writeProperties(out, properties);
   }
 
   @Override
@@ -210,9 +215,11 @@ implements DataStore<K, T> {
   }
 
   /**
-   * Returns the name of the schema to use for the persistent class. If the mapping schema name is
-   * provided it is returned first, else the properties file is searched, and the default schema name is
-   * returned if found. Else, the class name, without the package, of the persistent class is returned.
+   * Returns the name of the schema to use for the persistent class. 
+   * 
+   * First the schema name in the defined properties is returned. If null then
+   * the provided mappingSchemaName is returned. If this is null too,
+   * the class name, without the package, of the persistent class is returned.
    * @param mappingSchemaName the name of the schema as read from the mapping file
    * @param persistentClass persistent class
    */

Added: gora/trunk/gora-core/src/main/java/org/apache/gora/util/WritableUtils.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/util/WritableUtils.java?rev=1306866&view=auto
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/util/WritableUtils.java (added)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/util/WritableUtils.java Thu Mar 29 13:45:03 2012
@@ -0,0 +1,62 @@
+/**
+ * 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.util;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+import org.apache.hadoop.io.MapWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * An utility class for {@link Writable} related functionality.
+ */
+public class WritableUtils {
+  private WritableUtils() {
+    // prevents instantiation
+  }
+  
+  
+  public static final void writeProperties(DataOutput out, Properties props) throws IOException {
+    MapWritable propsWritable = new MapWritable();
+    for (Entry<Object, Object> prop : props.entrySet()) {
+      Writable key = new Text(prop.getKey().toString());
+      Writable value = new Text(prop.getValue().toString());
+      propsWritable.put(key,value);
+    }
+    propsWritable.write(out);
+  }
+  
+  public static final Properties readProperties(DataInput in) throws IOException {
+    Properties props = new Properties();
+    MapWritable propsWritable = new MapWritable();
+    propsWritable.readFields(in);
+    for (Entry<Writable, Writable> prop : propsWritable.entrySet()) {
+      String key = prop.getKey().toString();
+      String value = prop.getValue().toString();
+      props.put(key,value);
+    }
+    return props;
+  }
+
+}

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

Modified: gora/trunk/gora-core/src/test/java/org/apache/gora/GoraTestDriver.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/test/java/org/apache/gora/GoraTestDriver.java?rev=1306866&r1=1306865&r2=1306866&view=diff
==============================================================================
--- gora/trunk/gora-core/src/test/java/org/apache/gora/GoraTestDriver.java (original)
+++ gora/trunk/gora-core/src/test/java/org/apache/gora/GoraTestDriver.java Thu Mar 29 13:45:03 2012
@@ -56,7 +56,7 @@ public class GoraTestDriver {
    * method annotated with org.junit.BeforeClass
    */
   public void setUpClass() throws Exception {
-    setProperties(DataStoreFactory.properties);
+    setProperties(DataStoreFactory.createProps());
   }
 
   /** Should be called once after the tests have finished, probably in the
@@ -103,7 +103,7 @@ public class GoraTestDriver {
   @SuppressWarnings("unchecked")
   public<K, T extends Persistent> DataStore<K,T>
     createDataStore(Class<K> keyClass, Class<T> persistentClass) throws GoraException {
-    setProperties(DataStoreFactory.properties);
+    setProperties(DataStoreFactory.createProps());
     DataStore<K,T> dataStore = DataStoreFactory.createDataStore(
         (Class<? extends DataStore<K,T>>)dataStoreClass, keyClass, persistentClass, conf);
     dataStores.add(dataStore);

Modified: gora/trunk/gora-core/src/test/java/org/apache/gora/avro/mapreduce/TestDataFileAvroStoreMapReduce.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/test/java/org/apache/gora/avro/mapreduce/TestDataFileAvroStoreMapReduce.java?rev=1306866&r1=1306865&r2=1306866&view=diff
==============================================================================
--- gora/trunk/gora-core/src/test/java/org/apache/gora/avro/mapreduce/TestDataFileAvroStoreMapReduce.java (original)
+++ gora/trunk/gora-core/src/test/java/org/apache/gora/avro/mapreduce/TestDataFileAvroStoreMapReduce.java Thu Mar 29 13:45:03 2012
@@ -41,7 +41,7 @@ public class TestDataFileAvroStoreMapRed
   protected DataStore<String, WebPage> createWebPageDataStore() 
     throws IOException {
     DataFileAvroStore<String,WebPage> webPageStore = new DataFileAvroStore<String, WebPage>();
-    webPageStore.initialize(String.class, WebPage.class, DataStoreFactory.properties);
+    webPageStore.initialize(String.class, WebPage.class, DataStoreFactory.createProps());
     webPageStore.setOutputPath(WEBPAGE_OUTPUT);
     webPageStore.setInputPath(WEBPAGE_OUTPUT);
     

Modified: gora/trunk/gora-core/src/test/java/org/apache/gora/avro/store/TestAvroStore.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/test/java/org/apache/gora/avro/store/TestAvroStore.java?rev=1306866&r1=1306865&r2=1306866&view=diff
==============================================================================
--- gora/trunk/gora-core/src/test/java/org/apache/gora/avro/store/TestAvroStore.java (original)
+++ gora/trunk/gora-core/src/test/java/org/apache/gora/avro/store/TestAvroStore.java Thu Mar 29 13:45:03 2012
@@ -58,12 +58,12 @@ public class TestAvroStore {
   @Before
   public void setUp() throws Exception {
     employeeStore = createEmployeeDataStore();
-    employeeStore.initialize(String.class, Employee.class, DataStoreFactory.properties);
+    employeeStore.initialize(String.class, Employee.class, DataStoreFactory.createProps());
     employeeStore.setOutputPath(EMPLOYEE_OUTPUT);
     employeeStore.setInputPath(EMPLOYEE_OUTPUT);
 
     webPageStore = new AvroStore<String, WebPage>();
-    webPageStore.initialize(String.class, WebPage.class, DataStoreFactory.properties);
+    webPageStore.initialize(String.class, WebPage.class, DataStoreFactory.createProps());
     webPageStore.setOutputPath(WEBPAGE_OUTPUT);
     webPageStore.setInputPath(WEBPAGE_OUTPUT);
   }

Modified: gora/trunk/gora-core/src/test/java/org/apache/gora/store/TestDataStoreFactory.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/test/java/org/apache/gora/store/TestDataStoreFactory.java?rev=1306866&r1=1306865&r2=1306866&view=diff
==============================================================================
--- gora/trunk/gora-core/src/test/java/org/apache/gora/store/TestDataStoreFactory.java (original)
+++ gora/trunk/gora-core/src/test/java/org/apache/gora/store/TestDataStoreFactory.java Thu Mar 29 13:45:03 2012
@@ -70,7 +70,7 @@ public class TestDataStoreFactory {
     DataStore<?,?> dataStore3 = DataStoreFactory.getDataStore("org.apache.gora.mock.store.MockDataStore"
         , String.class, MockPersistent.class, conf);
     
-    Assert.assertTrue(dataStore1 == dataStore2);
+    Assert.assertNotSame(dataStore1, dataStore2);
     Assert.assertNotSame(dataStore1, dataStore3);
   }
   
@@ -85,7 +85,7 @@ public class TestDataStoreFactory {
   
   @Test
   public void testFindProperty() {
-    Properties properties = DataStoreFactory.properties;
+    Properties properties = DataStoreFactory.createProps();
     
     DataStore<String, MockPersistent> store = new DataFileAvroStore<String,MockPersistent>();
     

Added: gora/trunk/gora-core/src/test/java/org/apache/gora/util/TestWritableUtils.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/test/java/org/apache/gora/util/TestWritableUtils.java?rev=1306866&view=auto
==============================================================================
--- gora/trunk/gora-core/src/test/java/org/apache/gora/util/TestWritableUtils.java (added)
+++ gora/trunk/gora-core/src/test/java/org/apache/gora/util/TestWritableUtils.java Thu Mar 29 13:45:03 2012
@@ -0,0 +1,53 @@
+/**
+ * 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.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.util.Properties;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test case for {@link WritableUtils} class.
+ */
+public class TestWritableUtils {
+  @Test
+  public void testWritesReads() throws Exception {
+    Properties props = new Properties();
+    props.put("keyBlah", "valueBlah");
+    props.put("keyBlah2", "valueBlah2");
+    
+    ByteArrayOutputStream bos = new ByteArrayOutputStream();
+    DataOutput out = new DataOutputStream(bos);
+    WritableUtils.writeProperties(out, props);
+    
+    DataInput in = new DataInputStream(new ByteArrayInputStream(bos.toByteArray()));
+    
+    Properties propsRead = WritableUtils.readProperties(in);
+    
+    Assert.assertEquals(propsRead.get("keyBlah"), props.get("keyBlah"));
+    Assert.assertEquals(propsRead.get("keyBlah2"), props.get("keyBlah2"));
+  }
+}

Propchange: gora/trunk/gora-core/src/test/java/org/apache/gora/util/TestWritableUtils.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain