You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gora.apache.org by hs...@apache.org on 2012/11/16 07:20:30 UTC

svn commit: r1410209 - in /gora/trunk: ./ gora-core/src/main/java/org/apache/gora/mapreduce/ gora-core/src/main/java/org/apache/gora/query/ gora-core/src/main/java/org/apache/gora/query/impl/ gora-core/src/main/java/org/apache/gora/query/ws/impl/ gora-...

Author: hsaputra
Date: Fri Nov 16 06:20:28 2012
New Revision: 1410209

URL: http://svn.apache.org/viewvc?rev=1410209&view=rev
Log:
GORA-179 - Modify the Query interface to be Query<K, T extends Persistent> to be more precise

Modified:
    gora/trunk/CHANGES.txt
    gora/trunk/gora-core/src/main/java/org/apache/gora/mapreduce/GoraMapper.java
    gora/trunk/gora-core/src/main/java/org/apache/gora/query/PartitionQuery.java
    gora/trunk/gora-core/src/main/java/org/apache/gora/query/Query.java
    gora/trunk/gora-core/src/main/java/org/apache/gora/query/Result.java
    gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/PartitionQueryImpl.java
    gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java
    gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/ResultBase.java
    gora/trunk/gora-core/src/main/java/org/apache/gora/query/ws/impl/QueryWSBase.java
    gora/trunk/gora-core/src/main/java/org/apache/gora/store/DataStore.java
    gora/trunk/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBResult.java

Modified: gora/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/gora/trunk/CHANGES.txt?rev=1410209&r1=1410208&r2=1410209&view=diff
==============================================================================
--- gora/trunk/CHANGES.txt (original)
+++ gora/trunk/CHANGES.txt Fri Nov 16 06:20:28 2012
@@ -6,6 +6,8 @@ Gora Change Log
 
 trunk (current development)
 
+* GORA-179 Modify the Query interface to be Query<K, T extends Persistent> to be more precise
+
 * GORA-178 HBase fix ivy.xml to use the correct antconfig mapping (ferdy) 
 
 * GORA-172 java.lang.ClassNotFoundException: org.apache.gora.memory.store.MemStore.MemQuery (yumeng via lewismc)

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/mapreduce/GoraMapper.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/mapreduce/GoraMapper.java?rev=1410209&r1=1410208&r2=1410209&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/mapreduce/GoraMapper.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/mapreduce/GoraMapper.java Fri Nov 16 06:20:28 2012
@@ -29,8 +29,7 @@ import org.apache.hadoop.mapreduce.Parti
 /**
  * Base class for Gora based {@link Mapper}s.
  */
-public class GoraMapper<K1, V1 extends Persistent, K2, V2>
-  extends Mapper<K1, V1, K2, V2> {
+public class GoraMapper<K1, V1 extends Persistent, K2, V2> extends Mapper<K1, V1, K2, V2> {
 
   /**
    * Initializes the Mapper, and sets input parameters for the job. All of 
@@ -48,8 +47,7 @@ public class GoraMapper<K1, V1 extends P
    * @param reuseObjects whether to reuse objects in serialization
    */
   @SuppressWarnings("rawtypes")
-  public static <K1, V1 extends Persistent, K2, V2>
-  void initMapperJob(
+  public static <K1, V1 extends Persistent, K2, V2> void initMapperJob(
       Job job,
       Class<? extends DataStore<K1,V1>> dataStoreClass,
       Class<K1> inKeyClass, 
@@ -58,8 +56,7 @@ public class GoraMapper<K1, V1 extends P
       Class<V2> outValueClass,
       Class<? extends GoraMapper> mapperClass,
       Class<? extends Partitioner> partitionerClass, 
-      boolean reuseObjects)
-  throws IOException {
+      boolean reuseObjects) throws IOException {
     
     //set the input via GoraInputFormat
     GoraInputFormat.setInput(job, dataStoreClass, inKeyClass, inValueClass, reuseObjects);
@@ -88,8 +85,7 @@ public class GoraMapper<K1, V1 extends P
    * @param reuseObjects whether to reuse objects in serialization
    */  
   @SuppressWarnings("rawtypes")
-  public static <K1, V1 extends Persistent, K2, V2>
-  void initMapperJob(
+  public static <K1, V1 extends Persistent, K2, V2> void initMapperJob(
       Job job,
       Class<? extends DataStore<K1,V1>> dataStoreClass,
       Class<K1> inKeyClass, 
@@ -97,10 +93,9 @@ public class GoraMapper<K1, V1 extends P
       Class<K2> outKeyClass, 
       Class<V2> outValueClass,
       Class<? extends GoraMapper> mapperClass,
-      boolean reuseObjects)
-  throws IOException {
-    initMapperJob(job, dataStoreClass, inKeyClass, inValueClass, outKeyClass
-        , outValueClass, mapperClass, null, reuseObjects);
+      boolean reuseObjects) throws IOException {
+    initMapperJob(job, dataStoreClass, inKeyClass, inValueClass, outKeyClass,
+        outValueClass, mapperClass, null, reuseObjects);
   }
   
   /**
@@ -115,8 +110,7 @@ public class GoraMapper<K1, V1 extends P
    * @param reuseObjects whether to reuse objects in serialization
    */
   @SuppressWarnings("rawtypes")
-  public static <K1, V1 extends Persistent, K2, V2>
-  void initMapperJob(
+  public static <K1, V1 extends Persistent, K2, V2> void initMapperJob(
       Job job, 
       Query<K1,V1> query,
       DataStore<K1,V1> dataStore,
@@ -124,8 +118,7 @@ public class GoraMapper<K1, V1 extends P
       Class<V2> outValueClass,
       Class<? extends GoraMapper> mapperClass,
       Class<? extends Partitioner> partitionerClass, 
-      boolean reuseObjects)
-  throws IOException {
+      boolean reuseObjects) throws IOException {
     //set the input via GoraInputFormat
     GoraInputFormat.setInput(job, query, dataStore, reuseObjects);
 
@@ -148,15 +141,13 @@ public class GoraMapper<K1, V1 extends P
    * @param reuseObjects whether to reuse objects in serialization
    */
   @SuppressWarnings({ "rawtypes" })
-  public static <K1, V1 extends Persistent, K2, V2>
-  void initMapperJob(
+  public static <K1, V1 extends Persistent, K2, V2> void initMapperJob(
       Job job, 
       DataStore<K1,V1> dataStore,
       Class<K2> outKeyClass, 
       Class<V2> outValueClass,
       Class<? extends GoraMapper> mapperClass, 
-      boolean reuseObjects)
-  throws IOException {
+      boolean reuseObjects) throws IOException {
     initMapperJob(job, dataStore.newQuery(), dataStore, 
         outKeyClass, outValueClass, mapperClass, reuseObjects);
   }
@@ -172,17 +163,14 @@ public class GoraMapper<K1, V1 extends P
    * @param reuseObjects whether to reuse objects in serialization
    */
   @SuppressWarnings({ "rawtypes" })
-  public static <K1, V1 extends Persistent, K2, V2>
-  void initMapperJob(
+  public static <K1, V1 extends Persistent, K2, V2> void initMapperJob(
       Job job, 
       Query<K1,V1> query, 
       DataStore<K1,V1> dataStore,
       Class<K2> outKeyClass, 
       Class<V2> outValueClass,
       Class<? extends GoraMapper> mapperClass, 
-      boolean reuseObjects)
-  throws IOException {
-
+      boolean reuseObjects) throws IOException {
     initMapperJob(job, query, dataStore, outKeyClass, outValueClass,
         mapperClass, null, reuseObjects);
   }

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/query/PartitionQuery.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/query/PartitionQuery.java?rev=1410209&r1=1410208&r2=1410209&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/query/PartitionQuery.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/query/PartitionQuery.java Fri Nov 16 06:20:28 2012
@@ -18,12 +18,14 @@
 
 package org.apache.gora.query;
 
+import org.apache.gora.persistency.Persistent;
+
 /**
  * PartitionQuery divides the results of the Query to multi partitions, so that 
  * queries can be run locally on the nodes that hold the data. PartitionQuery's are 
  * used for generating Hadoop InputSplits.
  */
-public interface PartitionQuery<K, T> extends Query<K, T> {
+public interface PartitionQuery<K, T extends Persistent> extends Query<K, T> {
 
   /* PartitionQuery interface relaxes the dependency of DataStores to Hadoop*/
   

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/query/Query.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/query/Query.java?rev=1410209&r1=1410208&r2=1410209&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/query/Query.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/query/Query.java Fri Nov 16 06:20:28 2012
@@ -20,13 +20,14 @@ package org.apache.gora.query;
 
 import java.io.IOException;
 
+import org.apache.gora.persistency.Persistent;
 import org.apache.gora.store.DataStore;
 
 /**
  * A query to a data store to retrieve objects. Queries are constructed by 
  * the DataStore implementation via {@link DataStore#newQuery()}.
  */
-public interface Query<K, T> {
+public interface Query<K, T extends Persistent> {
 
   /**
    * Sets the dataStore of this query. Under normal operation, this call 
@@ -34,19 +35,19 @@ public interface Query<K, T> {
    * method only if you know what you are doing.
    * @param dataStore the dataStore of the query
    */
-  void setDataStore(DataStore<K,T> dataStore);
+  void setDataStore(DataStore<K, T> dataStore);
   
   /**
    * Returns the DataStore, that this Query is associated with.
    * @return the DataStore of the Query
    */
-  DataStore<K,T> getDataStore();
+  DataStore<K, T> getDataStore();
   
   /**
    * Executes the Query on the DataStore and returns the results.
    * @return the {@link Result} for the query.
    */
-  Result<K,T> execute() throws Exception, IOException;
+  Result<K, T> execute();
   
 //  /**
 //   * Compiles the query for performance and error checking. This 

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/query/Result.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/query/Result.java?rev=1410209&r1=1410208&r2=1410209&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/query/Result.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/query/Result.java Fri Nov 16 06:20:28 2012
@@ -20,6 +20,7 @@ package org.apache.gora.query;
 
 import java.io.IOException;
 
+import org.apache.gora.persistency.Persistent;
 import org.apache.gora.store.DataStore;
 
 /**
@@ -27,13 +28,13 @@ import org.apache.gora.store.DataStore;
  * iterated by calling {@link #next()}, {@link #get()} 
  * and {@link #getKey()}. 
  */
-public interface Result<K,T> {
+public interface Result<K, T extends Persistent> {
 
   /**
    * Returns the DataStore, that this Result is associated with.
    * @return the DataStore of the Result
    */
-  DataStore<K,T> getDataStore();
+  DataStore<K, T> getDataStore();
   
   /**
    * Returns the Query object for this Result.
@@ -80,7 +81,7 @@ public interface Result<K,T> {
   /**
    * Returns how far along the result has iterated, a value between 0 and 1.
    */
-  float getProgress() throws IOException, InterruptedException, Exception;
+  float getProgress() throws IOException, InterruptedException;
 
   void close() throws IOException;
   

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/PartitionQueryImpl.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/PartitionQueryImpl.java?rev=1410209&r1=1410208&r2=1410209&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/PartitionQueryImpl.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/PartitionQueryImpl.java Fri Nov 16 06:20:28 2012
@@ -33,8 +33,8 @@ import org.apache.gora.util.IOUtils;
 /**
  * Implementation for {@link PartitionQuery}.
  */
-public class PartitionQueryImpl<K, T extends PersistentBase>
-  extends QueryBase<K, T> implements PartitionQuery<K, T> {
+public class PartitionQueryImpl<K, T extends PersistentBase> extends QueryBase<K, T>
+    implements PartitionQuery<K, T> {
 
   protected Query<K, T> baseQuery;
   protected String[] locations;

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java?rev=1410209&r1=1410208&r2=1410209&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java Fri Nov 16 06:20:28 2012
@@ -42,8 +42,8 @@ import org.apache.hadoop.util.Reflection
 /**
  * Base class for Query implementations.
  */
-public abstract class QueryBase<K, T extends PersistentBase>  
-implements Query<K,T>, Writable, Configurable {
+public abstract class QueryBase<K, T extends PersistentBase>
+    implements Query<K,T>, Writable, Configurable {
 	
   protected DataStoreBase<K,T> dataStore;
 
@@ -60,8 +60,6 @@ implements Query<K,T>, Writable, Configu
 
   protected long limit = -1;
 
-  protected boolean isCompiled = false;
-
   private Configuration conf;
 
   public QueryBase(DataStore<K,T> dataStore) {
@@ -69,18 +67,10 @@ implements Query<K,T>, Writable, Configu
   }
 
   @Override
-  public Result<K,T> execute() throws Exception, IOException {
-    //compile();
+  public Result<K,T> execute() {
     return dataStore.execute(this);
   }
 
-//  @Override
-//  public void compile() {
-//    if(!isCompiled) {
-//      isCompiled = true;
-//    }
-//  }
-
   @Override
   public void setDataStore(DataStore<K, T> dataStore) {
     this.dataStore = (DataStoreBase<K, T>)dataStore;

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/ResultBase.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/ResultBase.java?rev=1410209&r1=1410208&r2=1410209&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/ResultBase.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/query/impl/ResultBase.java Fri Nov 16 06:20:28 2012
@@ -135,7 +135,5 @@ public abstract class ResultBase<K, T ex
   
   @Override
   public void close() throws IOException{
-	// TODO Auto-generated method stub
-	
   }
 }

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/query/ws/impl/QueryWSBase.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/query/ws/impl/QueryWSBase.java?rev=1410209&r1=1410208&r2=1410209&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/query/ws/impl/QueryWSBase.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/query/ws/impl/QueryWSBase.java Fri Nov 16 06:20:28 2012
@@ -29,7 +29,7 @@ import org.apache.gora.store.DataStore;
 /**
  * Base class for Query implementations.
  */
-public abstract class QueryWSBase<K, T extends Persistent> implements Query<K,T>{
+public abstract class QueryWSBase<K, T extends Persistent> implements Query<K,T> {
 	
   /**
    * Data store used for this query
@@ -88,7 +88,7 @@ public abstract class QueryWSBase<K, T e
   /**
    * Executes the query
    */
-  public Result<K,T> execute() throws Exception {
+  public Result<K,T> execute() {
     //compile();
     return dataStore.execute(this);
   }

Modified: gora/trunk/gora-core/src/main/java/org/apache/gora/store/DataStore.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-core/src/main/java/org/apache/gora/store/DataStore.java?rev=1410209&r1=1410208&r2=1410209&view=diff
==============================================================================
--- gora/trunk/gora-core/src/main/java/org/apache/gora/store/DataStore.java (original)
+++ gora/trunk/gora-core/src/main/java/org/apache/gora/store/DataStore.java Fri Nov 16 06:20:28 2012
@@ -42,7 +42,7 @@ import org.apache.gora.query.Result;
  * @param <K> the class of keys in the datastore
  * @param <T> the class of persistent objects in the datastore
  */
-public interface DataStore<K, T> {
+public interface DataStore<K, T extends Persistent> {
 
   /**
    * Initializes this DataStore.
@@ -51,8 +51,7 @@ public interface DataStore<K, T> {
    * @param properties extra metadata
    * @throws IOException
    */
-  void initialize(Class<K> keyClass, Class<T> persistentClass,
-      Properties properties);
+  void initialize(Class<K> keyClass, Class<T> persistentClass, Properties properties);
 
   /**
    * Sets the class of the keys
@@ -181,7 +180,7 @@ public interface DataStore<K, T> {
    * @return the results as a {@link Result} object.
    * @throws IOException
    */
-  Result<K,T> execute(Query<K, T> query);
+  Result<K, T> execute(Query<K, T> query);
 
   /**
    * Constructs and returns a new Query.
@@ -198,7 +197,7 @@ public interface DataStore<K, T> {
    * @return a List of PartitionQuery's
    * @throws IOException 
    */
-  List<PartitionQuery<K,T>> getPartitions(Query<K,T> query) throws IOException;
+  List<PartitionQuery<K, T>> getPartitions(Query<K, T> query) throws IOException;
 
   /**
    * Forces the write caches to be flushed. DataStore implementations may

Modified: gora/trunk/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBResult.java
URL: http://svn.apache.org/viewvc/gora/trunk/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBResult.java?rev=1410209&r1=1410208&r2=1410209&view=diff
==============================================================================
--- gora/trunk/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBResult.java (original)
+++ gora/trunk/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBResult.java Fri Nov 16 06:20:28 2012
@@ -64,7 +64,7 @@ public class DynamoDBResult<K, T extends
   /**
    * Gets the items reading progress
    */
-  public float getProgress() throws IOException, InterruptedException, Exception {
+  public float getProgress() throws IOException, InterruptedException {
     if (this.limit <= 0 || this.offset <= 0)
       return 0;
     return this.limit/this.offset;
@@ -82,7 +82,6 @@ public class DynamoDBResult<K, T extends
 
   @Override
   public void close() throws IOException {
-  // TODO Auto-generated method stub
   }
 
 }