You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@openjpa.apache.org by pp...@apache.org on 2010/05/12 23:08:35 UTC

svn commit: r943688 - in /openjpa/trunk/openjpa-slice/src: main/java/org/apache/openjpa/slice/ main/java/org/apache/openjpa/slice/jdbc/ test/java/org/apache/openjpa/slice/ test/java/org/apache/openjpa/slice/policy/

Author: ppoddar
Date: Wed May 12 21:08:34 2010
New Revision: 943688

URL: http://svn.apache.org/viewvc?rev=943688&view=rev
Log:
OPENJPA-1663,OPENJPA-1664: Add support to target find() or query to a subset of slices with plug-in policy. The existing, code intrusive mechanism of setting hints on query/fetch plan explicitly is retained. Intrusive, explicit setting of targets overrides policy based targets. 

Added:
    openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/FinderTargetPolicy.java   (with props)
    openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/QueryTargetPolicy.java   (with props)
    openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/TargetFetchConfiguration.java   (with props)
    openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQueryTargetPolicy.java   (with props)
    openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/policy/SampleQueryTargetPolicy.java   (with props)
Modified:
    openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedBroker.java
    openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedBrokerImpl.java
    openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedConfiguration.java
    openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedQueryImpl.java
    openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/DistributedJDBCConfigurationImpl.java
    openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/DistributedJDBCStoreManager.java
    openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestBasic.java
    openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQuery.java
    openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQueryMultiThreaded.java

Modified: openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedBroker.java
URL: http://svn.apache.org/viewvc/openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedBroker.java?rev=943688&r1=943687&r2=943688&view=diff
==============================================================================
--- openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedBroker.java (original)
+++ openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedBroker.java Wed May 12 21:08:34 2010
@@ -19,6 +19,7 @@
 package org.apache.openjpa.slice;
 
 import org.apache.openjpa.kernel.Broker;
+import org.apache.openjpa.slice.jdbc.TargetFetchConfiguration;
 
 /**
  * Extension to Broker to allow access to virtual data store.
@@ -31,4 +32,9 @@ public interface DistributedBroker exten
      * Gets the distributed store manager used by this receiver.
      */
     DistributedStoreManager getDistributedStoreManager();
+   
+    /**
+     * Gets the covariant fetch configuration that is aware of targets.
+     */
+    TargetFetchConfiguration getFetchConfiguration();
 }

Modified: openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedBrokerImpl.java
URL: http://svn.apache.org/viewvc/openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedBrokerImpl.java?rev=943688&r1=943687&r2=943688&view=diff
==============================================================================
--- openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedBrokerImpl.java (original)
+++ openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedBrokerImpl.java Wed May 12 21:08:34 2010
@@ -18,14 +18,18 @@
  */
 package org.apache.openjpa.slice;
 
+import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.openjpa.kernel.FetchConfiguration;
 import org.apache.openjpa.kernel.FinalizingBrokerImpl;
 import org.apache.openjpa.kernel.OpCallbacks;
 import org.apache.openjpa.kernel.OpenJPAStateManager;
 import org.apache.openjpa.kernel.QueryImpl;
 import org.apache.openjpa.kernel.StoreQuery;
 import org.apache.openjpa.lib.util.Localizer;
+import org.apache.openjpa.slice.jdbc.TargetFetchConfiguration;
+import org.apache.openjpa.util.OpenJPAId;
 
 /**
  * A specialized Broker to associate slice identifiers with the StateManagers as
@@ -38,98 +42,116 @@ import org.apache.openjpa.lib.util.Local
  * 
  */
 @SuppressWarnings("serial")
-public class DistributedBrokerImpl extends FinalizingBrokerImpl 
-    implements DistributedBroker {
-	private transient String _rootSlice;
-	private transient DistributedConfiguration _conf;
-	private final ReentrantSliceLock _lock;
-	
-	private static final Localizer _loc = Localizer.forPackage(DistributedBrokerImpl.class);
-
-	public DistributedBrokerImpl() {
-	    super();
-	    _lock = new ReentrantSliceLock();
-	}
-	
+public class DistributedBrokerImpl extends FinalizingBrokerImpl implements DistributedBroker {
+    private transient String _rootSlice;
+    private transient DistributedConfiguration _conf;
+    private final ReentrantSliceLock _lock;
+
+    private static final Localizer _loc = Localizer.forPackage(DistributedBrokerImpl.class);
+
+    public DistributedBrokerImpl() {
+        super();
+        _lock = new ReentrantSliceLock();
+    }
+
     public DistributedConfiguration getConfiguration() {
-    	if (_conf == null) {
-    		_conf = (DistributedConfiguration)super.getConfiguration();
-    	}
+        if (_conf == null) {
+            _conf = (DistributedConfiguration) super.getConfiguration();
+        }
         return _conf;
     }
-    
+
     public DistributedStoreManager getDistributedStoreManager() {
-        return (DistributedStoreManager)getStoreManager().getInnermostDelegate();
+        return (DistributedStoreManager) getStoreManager().getInnermostDelegate();
+    }
+
+    public TargetFetchConfiguration getFetchConfiguration() {
+        return (TargetFetchConfiguration) super.getFetchConfiguration();
     }
-    
-	/**
+
+    /**
      * Assigns slice identifier to the resultant StateManager as initialized by
-	 * the super class implementation. The slice identifier is decided by
-	 * {@link DistributionPolicy} for given <code>pc</code> if it is a root
+     * the super class implementation. The slice identifier is decided by
+     * {@link DistributionPolicy} for given <code>pc</code> if it is a root
      * instance i.e. the argument of the user application's persist() call. The
-	 * cascaded instances are detected by non-empty status of the current
-	 * operating set. The slice is assigned only if a StateManager has never
-	 * been assigned before.
-	 */
-	@Override
+     * cascaded instances are detected by non-empty status of the current
+     * operating set. The slice is assigned only if a StateManager has never
+     * been assigned before.
+     */
+    @Override
     public OpenJPAStateManager persist(Object pc, Object id, boolean explicit, OpCallbacks call) {
-		OpenJPAStateManager sm = getStateManager(pc);
-		SliceInfo info = null;
-        boolean replicated = SliceImplHelper.isReplicated(pc,
-                getConfiguration());
-        if (getOperatingSet().isEmpty()	&& !SliceImplHelper.isSliceAssigned(sm))
-        {
-            info = SliceImplHelper.getSlicesByPolicy(pc, getConfiguration(),
-				this);
-			_rootSlice = info.getSlices()[0]; 
-		}
-		sm = super.persist(pc, id, explicit, call);
-		if (!SliceImplHelper.isSliceAssigned(sm)) {
-			if (info == null) {
-			   info = replicated 
-               ? SliceImplHelper.getSlicesByPolicy(pc, getConfiguration(), this)
-			   : new SliceInfo(_rootSlice); 
-			}
-			info.setInto(sm);
-		}
-		return sm;
-	}
-
-	
-	@Override
-	public boolean endOperation() {
-	    try {
-	        return super.endOperation();
-	    } catch (Exception ex) {
-	        
-	    }
-	    return true;
-	}
-	
+        OpenJPAStateManager sm = getStateManager(pc);
+        SliceInfo info = null;
+        boolean replicated = SliceImplHelper.isReplicated(pc, getConfiguration());
+        if (getOperatingSet().isEmpty() && !SliceImplHelper.isSliceAssigned(sm)) {
+            info = SliceImplHelper.getSlicesByPolicy(pc, getConfiguration(), this);
+            _rootSlice = info.getSlices()[0];
+        }
+        sm = super.persist(pc, id, explicit, call);
+        if (!SliceImplHelper.isSliceAssigned(sm)) {
+            if (info == null) {
+                info = replicated ? SliceImplHelper.getSlicesByPolicy(pc, getConfiguration(), this) : new SliceInfo(
+                        _rootSlice);
+            }
+            info.setInto(sm);
+        }
+        return sm;
+    }
+
+    @Override
+    public boolean endOperation() {
+        try {
+            return super.endOperation();
+        } catch (Exception ex) {
+
+        }
+        return true;
+    }
+
     /**
      * Create a new query.
      */
     protected QueryImpl newQueryImpl(String lang, StoreQuery sq) {
         return new DistributedQueryImpl(this, lang, sq);
     }
-    
-	/**
-	 * Always uses lock irrespective of super's multi-threaded settings.
-	 */
+
+    /**
+     * Always uses lock irrespective of super's multi-threaded settings.
+     */
     @Override
     public void lock() {
         _lock.lock();
     }
-    
+
     @Override
     public void unlock() {
         _lock.unlock();
     }
-	
-	/**
-	 * A virtual datastore need not be opened.
-	 */
-	@Override
-	public void beginStore() {
-	}
+
+    /**
+     * A virtual datastore need not be opened.
+     */
+    @Override
+    public void beginStore() {
+    }
+
+    /**
+     * Overrides to target specific slices for find() calls.
+     */
+    @Override
+    public Object processArgument(Object oid) {
+        TargetFetchConfiguration fetch = getFetchConfiguration();
+        if (!fetch.isExplicitTarget()) {
+            FinderTargetPolicy policy = _conf.getFinderTargetPolicyInstance();
+            if (policy != null) {
+                if (oid instanceof OpenJPAId) {
+                    String[] targets = policy.getTargets(((OpenJPAId) oid).getType(), 
+                            ((OpenJPAId) oid).getIdObject(),
+                            _conf.getActiveSliceNames(), this);
+                    fetch.setTargets(targets);
+                }
+            }
+        }
+        return super.processArgument(oid);
+    }
 }

Modified: openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedConfiguration.java
URL: http://svn.apache.org/viewvc/openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedConfiguration.java?rev=943688&r1=943687&r2=943688&view=diff
==============================================================================
--- openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedConfiguration.java (original)
+++ openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedConfiguration.java Wed May 12 21:08:34 2010
@@ -118,5 +118,54 @@ public interface DistributedConfiguratio
      */
     void setReplicationPolicy(String policy);
     
+    /**
+     * Affirms if the given type is replicated.
+     */
     boolean isReplicated(Class<?> type);
+    
+    
+    /**
+     * Gets the policy that governs which slices will be targeted for a given query. 
+     */
+    QueryTargetPolicy getQueryTargetPolicyInstance();
+    
+    /**
+     * Gets the policy, as a plugin string, that governs which slices will be targeted 
+     * for a given query.
+     */
+    String getQueryTargetPolicy();
+    
+    /**
+     * Sets the policy that governs which slices will be targeted for a given query.
+     */
+    void setQueryTargetPolicyInstance(QueryTargetPolicy policy);
+    
+    /**
+     * Sets the policy, from the given plugin string, that governs which slices will 
+     * be targeted for a given query.
+     */
+    void setQueryTargetPolicy(String policy);
+
+    /**
+     * Gets the policy that governs which slices will be targeted for a given finder. 
+     */
+    FinderTargetPolicy getFinderTargetPolicyInstance();
+    
+    /**
+     * Gets the policy, as a plugin string, that governs which slices will be targeted 
+     * for a given finder.
+     */
+    String getFinderTargetPolicy();
+    
+    /**
+     * Sets the policy that governs which slices will be targeted for a given finder.
+     */
+    void setFinderTargetPolicyInstance(FinderTargetPolicy policy);
+    
+    /**
+     * Sets the policy, from the given plugin string, that governs which slices will 
+     * be targeted for a given finder.
+     */
+    void setFinderTargetPolicy(String policy);
+    
 }

Modified: openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedQueryImpl.java
URL: http://svn.apache.org/viewvc/openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedQueryImpl.java?rev=943688&r1=943687&r2=943688&view=diff
==============================================================================
--- openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedQueryImpl.java (original)
+++ openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/DistributedQueryImpl.java Wed May 12 21:08:34 2010
@@ -18,9 +18,13 @@
  */
 package org.apache.openjpa.slice;
 
+import java.util.Collections;
+import java.util.Map;
+
 import org.apache.openjpa.kernel.Broker;
 import org.apache.openjpa.kernel.QueryImpl;
 import org.apache.openjpa.kernel.StoreQuery;
+import org.apache.openjpa.slice.jdbc.TargetFetchConfiguration;
 
 /**
  * Extension with slice locking policy.
@@ -31,12 +35,33 @@ import org.apache.openjpa.kernel.StoreQu
 @SuppressWarnings("serial")
 public class DistributedQueryImpl extends QueryImpl {
     private final ReentrantSliceLock _lock;
+    private DistributedConfiguration _conf;
+    
     public DistributedQueryImpl(Broker broker, String language, StoreQuery storeQuery) {
         super(broker, language, storeQuery);
         _lock = new ReentrantSliceLock();
+        _conf = (DistributedConfiguration)broker.getConfiguration();
     }
     
     /**
+     * Overrides to set the query targets via policy, if user has not already set the targets via hint
+     * explicitly on this query. 
+     */
+    @Override
+    public Object execute(Map params) {
+        TargetFetchConfiguration fetch = (TargetFetchConfiguration)getFetchConfiguration();
+        if (!fetch.isExplicitTarget()) {
+            QueryTargetPolicy policy = _conf.getQueryTargetPolicyInstance();
+            if (policy != null) {
+                String[] targets = policy.getTargets(getQueryString(), Collections.unmodifiableMap(params), 
+                        getLanguage(), _conf.getActiveSliceNames(), this.getBroker());
+                fetch.setTargets(targets);
+            }
+        }
+        return super.execute(params);
+    }
+
+    /**
      * Always uses lock irrespective of super's multi-threaded settings.
      */
     @Override

Added: openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/FinderTargetPolicy.java
URL: http://svn.apache.org/viewvc/openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/FinderTargetPolicy.java?rev=943688&view=auto
==============================================================================
--- openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/FinderTargetPolicy.java (added)
+++ openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/FinderTargetPolicy.java Wed May 12 21:08:34 2010
@@ -0,0 +1,50 @@
+/*
+ * 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.openjpa.slice;
+
+import java.util.List;
+
+/**
+ * Policy to select one or more of the physical databases referred as 
+ * <em>slice</em> in which a given finder will be executed.
+ * 
+ * @author Pinaki Poddar 
+ * 
+ * @see DistributionPolicy
+ *
+ */
+public interface FinderTargetPolicy {
+	/**
+     * Gets the name of the slices where a given finder will be executed.
+	 *  
+	 * @param cls The class of the finder. 
+	 * @param oid the primary key for the finder
+	 * @param slices list of names of the active slices. The ordering of 
+	 * the list is either explicit <code>openjpa.slice.Names</code> property
+	 * or implicit i.e. alphabetic order of available identifiers if 
+	 * <code>openjpa.slice.Names</code> is unspecified.  
+     * @param context generic persistence context managing the given instance.
+	 * 
+	 * @return identifier of the slices. This names must match one of the
+	 * given slice names. 
+	 *  
+	 * @see DistributedConfiguration#getActiveSliceNames()
+	 */
+	String[] getTargets(Class<?> cls, Object oid, List<String> slices, Object context);
+}

Propchange: openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/FinderTargetPolicy.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/QueryTargetPolicy.java
URL: http://svn.apache.org/viewvc/openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/QueryTargetPolicy.java?rev=943688&view=auto
==============================================================================
--- openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/QueryTargetPolicy.java (added)
+++ openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/QueryTargetPolicy.java Wed May 12 21:08:34 2010
@@ -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.openjpa.slice;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Policy to select one or more of the physical databases referred as 
+ * <em>slice</em> in which a given query will be executed.
+ * 
+ * @author Pinaki Poddar 
+ * 
+ * @see DistributionPolicy
+ *
+ */
+public interface QueryTargetPolicy {
+	/**
+     * Gets the name of the slices where a given query will be executed.
+	 *  
+	 * @param query The query string to be executed. 
+	 * @param params the bound parameters of the query
+	 * @param language the query language
+	 * @param slices list of names of the active slices. The ordering of 
+	 * the list is either explicit <code>openjpa.slice.Names</code> property
+	 * or implicit i.e. alphabetic order of available identifiers if 
+	 * <code>openjpa.slice.Names</code> is unspecified.  
+     * @param context generic persistence context managing the given instance.
+	 * 
+	 * @return identifier of the slices. This names must match one of the
+	 * given slice names. 
+	 *  
+	 * @see DistributedConfiguration#getActiveSliceNames()
+	 */
+	String[] getTargets(String query, Map<Object,Object> params, String language,
+	        List<String> slices, Object context);
+}

Propchange: openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/QueryTargetPolicy.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/DistributedJDBCConfigurationImpl.java
URL: http://svn.apache.org/viewvc/openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/DistributedJDBCConfigurationImpl.java?rev=943688&r1=943687&r2=943688&view=diff
==============================================================================
--- openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/DistributedJDBCConfigurationImpl.java (original)
+++ openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/DistributedJDBCConfigurationImpl.java Wed May 12 21:08:34 2010
@@ -51,7 +51,9 @@ import org.apache.openjpa.meta.ClassMeta
 import org.apache.openjpa.meta.MetaDataRepository;
 import org.apache.openjpa.slice.DistributedBrokerImpl;
 import org.apache.openjpa.slice.DistributionPolicy;
+import org.apache.openjpa.slice.FinderTargetPolicy;
 import org.apache.openjpa.slice.ProductDerivation;
+import org.apache.openjpa.slice.QueryTargetPolicy;
 import org.apache.openjpa.slice.ReplicationPolicy;
 import org.apache.openjpa.slice.Slice;
 import org.apache.openjpa.util.UserException;
@@ -77,6 +79,8 @@ public class DistributedJDBCConfiguratio
     protected StringListValue namesPlugin;
     public PluginValue distributionPolicyPlugin;
     public PluginValue replicationPolicyPlugin;
+    public PluginValue queryTargetPolicyPlugin;
+    public PluginValue finderTargetPolicyPlugin;
     public StringListValue replicatedTypesPlugin;
     
     private ReplicatedTypeRepository _replicationRepos;
@@ -107,6 +111,12 @@ public class DistributedJDBCConfiguratio
         replicationPolicyPlugin.setString("all");
         replicationPolicyPlugin.setDynamic(true);
         
+        queryTargetPolicyPlugin = addPlugin(PREFIX_SLICE + "QueryTargetPolicy", true);
+        queryTargetPolicyPlugin.setDynamic(true);
+        
+        finderTargetPolicyPlugin = addPlugin(PREFIX_SLICE + "FinderTargetPolicy", true);
+        finderTargetPolicyPlugin.setDynamic(true);
+        
         replicatedTypesPlugin = new StringListValue(PREFIX_SLICE + "ReplicatedTypes");
         addValue(replicatedTypesPlugin);
         
@@ -242,6 +252,54 @@ public class DistributedJDBCConfiguratio
         replicationPolicyPlugin.setString(policy);
     }
 
+    public QueryTargetPolicy getQueryTargetPolicyInstance() {
+        if (queryTargetPolicyPlugin.get() == null) {
+            queryTargetPolicyPlugin.instantiate(ReplicationPolicy.class,
+                    this, true);
+        }
+        return (QueryTargetPolicy) queryTargetPolicyPlugin.get();
+    }
+    
+    public String getQueryTargetPolicy() {
+        if (queryTargetPolicyPlugin.get() == null) {
+            queryTargetPolicyPlugin.instantiate(QueryTargetPolicy.class,
+                    this, true);
+        }
+        return queryTargetPolicyPlugin.getString();
+    }
+
+    public void setQueryTargetPolicyInstance(QueryTargetPolicy policy) {
+        queryTargetPolicyPlugin.set(policy);
+    }
+    
+    public void setQueryTargetPolicy(String policy) {
+        queryTargetPolicyPlugin.setString(policy);
+    }
+    
+    public FinderTargetPolicy getFinderTargetPolicyInstance() {
+        if (finderTargetPolicyPlugin.get() == null) {
+            finderTargetPolicyPlugin.instantiate(ReplicationPolicy.class,
+                    this, true);
+        }
+        return (FinderTargetPolicy) finderTargetPolicyPlugin.get();
+    }
+    
+    public String getFinderTargetPolicy() {
+        if (finderTargetPolicyPlugin.get() == null) {
+            finderTargetPolicyPlugin.instantiate(FinderTargetPolicy.class,
+                    this, true);
+        }
+        return finderTargetPolicyPlugin.getString();
+    }
+
+    public void setFinderTargetPolicyInstance(FinderTargetPolicy policy) {
+        finderTargetPolicyPlugin.set(policy);
+    }
+    
+    public void setFinderTargetPolicy(String policy) {
+        finderTargetPolicyPlugin.setString(policy);
+    }
+
     public DistributedDataSource getConnectionFactory() {
         if (virtualDataSource == null) {
             virtualDataSource = createDistributedDataStore();

Modified: openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/DistributedJDBCStoreManager.java
URL: http://svn.apache.org/viewvc/openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/DistributedJDBCStoreManager.java?rev=943688&r1=943687&r2=943688&view=diff
==============================================================================
--- openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/DistributedJDBCStoreManager.java (original)
+++ openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/DistributedJDBCStoreManager.java Wed May 12 21:08:34 2010
@@ -25,6 +25,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.BitSet;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -37,6 +38,7 @@ import java.util.concurrent.Future;
 import org.apache.openjpa.enhance.PersistenceCapable;
 import org.apache.openjpa.jdbc.conf.JDBCConfiguration;
 import org.apache.openjpa.jdbc.kernel.ConnectionInfo;
+import org.apache.openjpa.jdbc.kernel.JDBCFetchConfigurationImpl;
 import org.apache.openjpa.jdbc.kernel.JDBCStore;
 import org.apache.openjpa.jdbc.kernel.JDBCStoreManager;
 import org.apache.openjpa.jdbc.sql.Result;
@@ -62,6 +64,7 @@ import org.apache.openjpa.slice.ProductD
 import org.apache.openjpa.slice.Slice;
 import org.apache.openjpa.slice.SliceImplHelper;
 import org.apache.openjpa.slice.SliceInfo;
+import org.apache.openjpa.slice.SlicePersistence;
 import org.apache.openjpa.slice.SliceThread;
 import org.apache.openjpa.util.InternalException;
 import org.apache.openjpa.util.StoreException;
@@ -441,6 +444,12 @@ class DistributedJDBCStoreManager extend
         }
         return ret;
     }
+    
+    @Override
+    public FetchConfiguration newFetchConfiguration() {
+        return new TargetFetchConfiguration();
+    }
+
 
     /**
      * Sets the context for this receiver and all its underlying slices.
@@ -481,7 +490,7 @@ class DistributedJDBCStoreManager extend
     
     /**
      * Gets the list of slices mentioned as  
-     * {@link ProductDerivation#HINT_TARGET hint} of the given
+     * {@link SlicePersistence#HINT_TARGET hint} of the given
      * {@link FetchConfiguration#getHint(String) fetch configuration}. 
      * 
      * @return all active slices if a) the hint is not specified or b) a null 
@@ -490,15 +499,19 @@ class DistributedJDBCStoreManager extend
     List<SliceStoreManager> getTargets(FetchConfiguration fetch) {
         if (fetch == null)
             return _slices;
-        Object hint = fetch.getHint(ProductDerivation.HINT_TARGET);
-        if (hint == null || !(hint instanceof String)) 
+        Object hint = fetch.getHint(SlicePersistence.HINT_TARGET);
+        if (hint == null || !(hint instanceof String || hint instanceof String[])) 
             return _slices;
-        List<String> targetNames = Arrays.asList(hint.toString().split("\\,"));
+        String[] targetNames = hint instanceof String 
+                ? new String[]{hint.toString()} : (String[])hint;
         List<SliceStoreManager> targets = new ArrayList<SliceStoreManager>();
         for (SliceStoreManager slice : _slices) {
-           if (targetNames.contains(slice.getName()))
-              targets.add(slice);
-           }
+            for (String name : targetNames) {
+                if (slice.getName().equals(name)) {
+                    targets.add(slice);
+                }
+            }
+        }
         if (targets.isEmpty())
             return _slices;
         return targets;

Added: openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/TargetFetchConfiguration.java
URL: http://svn.apache.org/viewvc/openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/TargetFetchConfiguration.java?rev=943688&view=auto
==============================================================================
--- openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/TargetFetchConfiguration.java (added)
+++ openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/TargetFetchConfiguration.java Wed May 12 21:08:34 2010
@@ -0,0 +1,90 @@
+package org.apache.openjpa.slice.jdbc;
+
+import org.apache.openjpa.jdbc.kernel.JDBCFetchConfigurationImpl;
+import org.apache.openjpa.kernel.FetchConfiguration;
+import org.apache.openjpa.slice.SlicePersistence;
+import org.apache.openjpa.util.InternalException;
+import org.apache.openjpa.util.UserException;
+
+/**
+ * A fetch configuration that is aware of special hint to narrow its operation on 
+ * subset of slices.
+ * 
+ * @author Pinaki Poddar
+ *
+ */
+@SuppressWarnings("serial")
+public class TargetFetchConfiguration extends JDBCFetchConfigurationImpl implements FetchConfiguration {
+    boolean _explicitTarget = false;
+    
+    public TargetFetchConfiguration() {
+        super();
+    }
+    
+    /**
+     * Setting hints on this configuration is treated specially if the given key
+     * is {@linkplain SlicePersistence#HINT_TARGET a target hint}.
+     *  
+     * @param value if the given key is target hint, then the value can be either
+     * null, a String or a non-zero sized String array. It can not be a zero-sized
+     * String array.
+     */
+    @Override
+    public void setHint(String key, Object value) {
+        super.setHint(key, value);
+        _explicitTarget = SlicePersistence.HINT_TARGET.equals(key);
+    }
+
+    public void setHint(String key, Object value, Object original) {
+        super.setHint(key, value, original);
+        _explicitTarget = SlicePersistence.HINT_TARGET.equals(key);
+    }
+       
+    public void setTargets(String[] targets) {
+        super.setHint(SlicePersistence.HINT_TARGET, targets);
+        _explicitTarget = false;
+    }
+    
+    /**
+     * Affirms if the target is set on this receiver explicitly (i.e. by the user).
+     */
+    public boolean isExplicitTarget() {
+        return _explicitTarget;
+    }
+    
+    String[] toSliceNames(Object o, boolean user) {
+        if (o == null)
+            return null;
+        if (o instanceof String) {
+            return new String[]{o.toString()};
+        }
+        if (o instanceof String[]) {
+            if (((String[])o).length == 0) {
+                throw new InternalException("Hint values " + o + " are wrong type " + o.getClass());
+                
+            }
+            return (String[])o;
+        }
+        throw new InternalException("Hint values " + o + " are wrong type " + o.getClass());
+    }
+    
+    void assertTargets(String[] targets, boolean user) {
+        if (targets != null && targets.length == 0) {
+            if (user) {
+                throw new UserException("Hint values " + targets + " are empty");
+            }
+        }
+    }
+    
+    protected TargetFetchConfiguration newInstance(ConfigurationState state) {
+        JDBCConfigurationState jstate = (state == null) ? null : _state;
+        return new TargetFetchConfiguration(state, jstate);
+    }
+    
+    protected TargetFetchConfiguration(ConfigurationState state, 
+            JDBCConfigurationState jstate) {
+            super(state, jstate);
+        }
+
+
+}

Propchange: openjpa/trunk/openjpa-slice/src/main/java/org/apache/openjpa/slice/jdbc/TargetFetchConfiguration.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestBasic.java
URL: http://svn.apache.org/viewvc/openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestBasic.java?rev=943688&r1=943687&r2=943688&view=diff
==============================================================================
--- openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestBasic.java (original)
+++ openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestBasic.java Wed May 12 21:08:34 2010
@@ -284,7 +284,7 @@ public class TestBasic extends SliceTest
         for (String hint : hints) {
             em.clear();
             Query query = em.createQuery(jpql).setParameter("name", "India");
-            query.setHint(ProductDerivation.HINT_TARGET, hint);
+            query.setHint(SlicePersistence.HINT_TARGET, hint);
             india = (Country)query.getSingleResult();
             assertEquals(india.getPopulation(), 1201);
             assertTrue(SlicePersistence.isReplicated(india));

Modified: openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQuery.java
URL: http://svn.apache.org/viewvc/openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQuery.java?rev=943688&r1=943687&r2=943688&view=diff
==============================================================================
--- openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQuery.java (original)
+++ openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQuery.java Wed May 12 21:08:34 2010
@@ -196,7 +196,22 @@ public class TestQuery extends SliceTest
         EntityManager em = emf.createEntityManager();
         em.getTransaction().begin();
         Query query = em.createQuery("SELECT p FROM PObject p");
-        query.setHint(ProductDerivation.HINT_TARGET, "Even");
+        query.setHint(SlicePersistence.HINT_TARGET, "Even");
+        List result = query.getResultList();
+        for (Object pc : result) {
+            String slice = SlicePersistence.getSlice(pc);
+            assertTrue("Expected original slice " + slice + " in " + targets, targets.contains(slice));
+        }
+        em.getTransaction().rollback();
+    }
+    
+    public void testQueryTargetPolicy() {
+        List<String> targets = new ArrayList<String>();
+        targets.add("Even");
+        EntityManager em = emf.createEntityManager();
+        em.getTransaction().begin();
+        Query query = em.createQuery("SELECT p FROM PObject p");
+        query.setHint(SlicePersistence.HINT_TARGET, "Even");
         List result = query.getResultList();
         for (Object pc : result) {
             String slice = SlicePersistence.getSlice(pc);

Modified: openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQueryMultiThreaded.java
URL: http://svn.apache.org/viewvc/openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQueryMultiThreaded.java?rev=943688&r1=943687&r2=943688&view=diff
==============================================================================
--- openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQueryMultiThreaded.java (original)
+++ openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQueryMultiThreaded.java Wed May 12 21:08:34 2010
@@ -34,6 +34,7 @@ import java.util.concurrent.TimeoutExcep
 
 import javax.persistence.EntityManager;
 import javax.persistence.Query;
+import javax.persistence.TypedQuery;
 
 /**
  * Tests when multiple user threads enter the same EntityManager and executes 
@@ -252,7 +253,7 @@ public class TestQueryMultiThreaded exte
             futures[i] = group.submit(new Callable<Object>() {
 
                 public Object call() {
-                    query.setHint(ProductDerivation.HINT_TARGET, "Even");
+                    query.setHint(SlicePersistence.HINT_TARGET, "Even");
                     List result = query.getResultList();
                     for (Object pc : result) {
                         String slice = SlicePersistence.getSlice(pc);
@@ -303,30 +304,22 @@ public class TestQueryMultiThreaded exte
         em.getTransaction().rollback();
     }
 
-    /**
-     * This test is currently retired.
-     * 
-     * @see <A HREF="https://issues.apache.org/jira/browse/OPENJPA-1044">
-     * OPENJPA-1044</A>
-     * for details.
-     */
     public void testQueryParameterEntity() {
         final EntityManager em = emf.createEntityManager();
         em.getTransaction().begin();
-        final Query addressQ =
-            em.createQuery("select a from Address a where a.city = :city");
+        final TypedQuery<Address> addressQ = em.createQuery(
+                "select a from Address a where a.city = :city", Address.class);
 
-        final Query personQ =
-            em.createQuery("SELECT p FROM Person p WHERE p.address = :a");
+        final TypedQuery<Person> personQ = em.createQuery(
+                "SELECT p FROM Person p WHERE p.address = :a", Person.class);
         for (int i = 0; i < THREADS; i++) {
             futures[i] = group.submit(new Callable<Object>() {
                 public Object call() {
-                    Address a = (Address) addressQ.setParameter("city", "Rome")
+                    Address a = addressQ.setParameter("city", "Rome")
                         .getSingleResult();
                     assertNotNull(a);
                     assertEquals("Odd", SlicePersistence.getSlice(a));
-                    List<Person> result =
-                        personQ.setParameter("a", a).getResultList();
+                    List<Person> result = personQ.setParameter("a", a).getResultList();
                     assertEquals(1, result.size());
                     Person p = result.get(0);
                     assertEquals("Odd", SlicePersistence.getSlice(p));

Added: openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQueryTargetPolicy.java
URL: http://svn.apache.org/viewvc/openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQueryTargetPolicy.java?rev=943688&view=auto
==============================================================================
--- openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQueryTargetPolicy.java (added)
+++ openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQueryTargetPolicy.java Wed May 12 21:08:34 2010
@@ -0,0 +1,129 @@
+/*
+ * 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.openjpa.slice;
+
+import java.util.List;
+
+import javax.persistence.EntityManager;
+import javax.persistence.Query;
+
+/**
+ * Tests query ordering.
+ * 
+ * @author Pinaki Poddar 
+ *
+ */
+public class TestQueryTargetPolicy extends SliceTestCase {
+
+    private int POBJECT_COUNT = 2;
+    private int VALUE_MIN = 100;
+    
+    protected String getPersistenceUnitName() {
+        return "ordering";
+    }
+
+    public void setUp() throws Exception {
+        super.setUp(PObject.class, Person.class, Address.class, Country.class,
+                Car.class, Manufacturer.class,
+                "openjpa.slice.QueryTargetPolicy", 
+                "org.apache.openjpa.slice.policy.SampleQueryTargetPolicy",
+        		CLEAR_TABLES);
+        int count = count(PObject.class);
+        if (count == 0) {
+            create(POBJECT_COUNT);
+        }
+    }
+    
+    
+    
+    void create(int N) {
+        EntityManager em = emf.createEntityManager();
+        em.getTransaction().begin();
+        for (int i=0;i < POBJECT_COUNT;i++) {
+            PObject pc = new PObject();
+            pc.setValue(VALUE_MIN + i);
+            em.persist(pc);
+            String slice = SlicePersistence.getSlice(pc);
+            String expected = (pc.getValue()%2 == 0) ? "Even" : "Odd";
+            assertEquals(expected, slice);
+        }
+        Person p1 = new Person();
+        Person p2 = new Person();
+        Address a1 = new Address();
+        Address a2 = new Address();
+        p1.setName("Even");
+        p2.setName("Odd");
+        a1.setCity("San Francisco");
+        a2.setCity("Rome");
+        p1.setAddress(a1);
+        p2.setAddress(a2);
+        em.persist(p1);
+        em.persist(p2);
+        assertEquals("Even", SlicePersistence.getSlice(p1));
+        assertEquals("Odd", SlicePersistence.getSlice(p2));
+        
+        em.getTransaction().commit();
+    }
+    
+    public static final String QueryPersonByName = "select p from Person p where p.name=:name";
+    public static final String QueryPersonByNameSwap = "select q from Person q where q.name=:name";
+    
+    public void testTargetSingleSlice() {
+        EntityManager em = emf.createEntityManager();
+        em.getTransaction().begin();
+        Query q1 = em.createQuery(QueryPersonByName);
+        List<?> result1 = q1.setParameter("name", "Even").getResultList();
+        String[] targets1 = getTargetSlices(q1);
+        assertArrayEquals(new String[]{"Even"}, targets1);
+        assertFalse(result1.isEmpty());
+
+        Query q2 = em.createQuery(QueryPersonByNameSwap);
+        List<?> result2 = q2.setParameter("name", "Even").getResultList();
+        String[] targets2 = getTargetSlices(q2);
+        assertArrayEquals(new String[]{"Odd"}, targets2);
+        assertTrue(result2.isEmpty());
+    }
+    
+    <T> void assertArrayEquals(T[] a, T[] b) {
+        assertEquals(a.length, b.length);
+        for (int i = 0; i < a.length; i++)
+            assertEquals(a[i], b[i]);
+    }
+    
+    
+    /**
+     * Gets the slice names on which the given query is targeted. 
+     * @param q
+     * @return
+     */
+    public static String[] getTargetSlices(Query q) {
+        Object targets = q.unwrap(org.apache.openjpa.kernel.Query.class)
+         .getFetchConfiguration()
+         .getHint(SlicePersistence.HINT_TARGET);
+        if (targets == null)
+            return null;
+        if (targets instanceof String) 
+            return new String[]{targets.toString()};
+        if (targets instanceof String[]) {
+            return (String[])targets;
+        }
+        return null;
+    }    
+    
+}

Propchange: openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/TestQueryTargetPolicy.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/policy/SampleQueryTargetPolicy.java
URL: http://svn.apache.org/viewvc/openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/policy/SampleQueryTargetPolicy.java?rev=943688&view=auto
==============================================================================
--- openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/policy/SampleQueryTargetPolicy.java (added)
+++ openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/policy/SampleQueryTargetPolicy.java Wed May 12 21:08:34 2010
@@ -0,0 +1,47 @@
+/*
+ * 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.openjpa.slice.policy;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.openjpa.slice.QueryTargetPolicy;
+import org.apache.openjpa.slice.TestQueryTargetPolicy;
+
+public class SampleQueryTargetPolicy implements QueryTargetPolicy {
+    @Override
+    public String[] getTargets(String query, Map<Object, Object> params, 
+            String language, List<String> slices,
+            Object context) {
+        if (TestQueryTargetPolicy.QueryPersonByName.equals(query)) {
+            if ("Even".equals(params.get("name")))
+                    return new String[]{"Even"};
+            if ("Odd".equals(params.get("name")))
+                return new String[]{"Odd"};
+        }
+        if (TestQueryTargetPolicy.QueryPersonByNameSwap.equals(query)) {
+            if ("Even".equals(params.get("name")))
+                    return new String[]{"Odd"};
+            if ("Odd".equals(params.get("name")))
+                return new String[]{"Even"};
+        }
+        return slices.toArray(new String[slices.size()]);
+    }
+
+}

Propchange: openjpa/trunk/openjpa-slice/src/test/java/org/apache/openjpa/slice/policy/SampleQueryTargetPolicy.java
------------------------------------------------------------------------------
    svn:eol-style = native