You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ds...@apache.org on 2016/06/07 01:04:29 UTC

[1/5] incubator-geode git commit: GEODE-1464: remove sqlf code

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 6967ac19f -> 880f86483


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySender.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySender.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySender.java
index 0704bb8..7e2a0af 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySender.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySender.java
@@ -908,7 +908,7 @@ public abstract class AbstractGatewaySender implements GatewaySender,
       }
     } else {
       GatewaySenderEventCallbackArgument geCallbackArg = new GatewaySenderEventCallbackArgument(
-          callbackArg, this.getMyDSId(), allRemoteDSIds, true);
+          callbackArg, this.getMyDSId(), allRemoteDSIds);
       clonedEvent.setCallbackArgument(geCallbackArg);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySenderEventProcessor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySenderEventProcessor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySenderEventProcessor.java
index c57aebc..ce08e8d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySenderEventProcessor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySenderEventProcessor.java
@@ -810,7 +810,7 @@ public abstract class AbstractGatewaySenderEventProcessor extends Thread {
           }
           GatewaySenderEventCallbackArgument geCallbackArg = new GatewaySenderEventCallbackArgument(
               event.getRawCallbackArgument(), this.sender.getMyDSId(),
-              allRemoteDSIds, true);
+              allRemoteDSIds);
           event.setCallbackArgument(geCallbackArg);
           GatewaySenderEventImpl pdxSenderEvent = new GatewaySenderEventImpl(
               EnumListenerEvent.AFTER_UPDATE, event, null); // OFFHEAP: event for pdx type meta data so it should never be off-heap

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventCallbackArgument.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventCallbackArgument.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventCallbackArgument.java
index f84dbbb..82476c8 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventCallbackArgument.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventCallbackArgument.java
@@ -96,14 +96,10 @@ WrappedCallbackArgument implements DataSerializableFixedID {
    * @param originalReceivers
    *          The list of <code>Gateway</code> s to which the event has been
    *          originally sent
-   * @param serializeCBArg
-   *          boolean indicating whether to serialize callback argument
-   * 
    */
   public GatewaySenderEventCallbackArgument(Object originalCallbackArg,
-      int originatingDSId, List<Integer> originalReceivers,
-      boolean serializeCBArg) {
-    super(originalCallbackArg, serializeCBArg);
+      int originatingDSId, List<Integer> originalReceivers) {
+    super(originalCallbackArg);
     this.originatingDSId = originatingDSId;
     initializeReceipientDSIds(originalReceivers);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
index 8a811e2..83811ec 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderEventImpl.java
@@ -515,10 +515,6 @@ public class GatewaySenderEventImpl implements
    *  //OFFHEAP TODO: Optimize callers by returning a reference to the off heap value
    */
   public Object getValue() {
-    if (CachedDeserializableFactory.preferObject()) {
-      // sqlf does not use CacheDeserializable wrappers
-      return getDeserializedValue();
-    }
     Object rawValue = this.value;
     if (rawValue == null) {
       rawValue = this.substituteValue;
@@ -535,9 +531,6 @@ public class GatewaySenderEventImpl implements
     if (valueIsObject == 0x00) {
       //if the value is a byte array, just return it
       return rawValue;
-    } else if (CachedDeserializableFactory.preferObject()) {
-      // sqlf does not use CacheDeserializable wrappers
-      return rawValue;
     } else if (rawValue instanceof byte[]) {
       return CachedDeserializableFactory.create((byte[]) rawValue);
     } else {
@@ -947,9 +940,7 @@ public class GatewaySenderEventImpl implements
      */
     @Retained(OffHeapIdentifier.GATEWAY_SENDER_EVENT_IMPL_VALUE)
     StoredObject so = null;
-    if (event.hasDelta()) {
-      this.valueIsObject = 0x02;
-    } else {
+    {
       ReferenceCountHelper.setReferenceCountOwner(this);
       so = event.getOffHeapNewValue();
       ReferenceCountHelper.setReferenceCountOwner(null);      
@@ -969,7 +960,7 @@ public class GatewaySenderEventImpl implements
       // can share a reference to the off-heap value.
       this.value = event.getCachedSerializedNewValue();
     } else {
-      final Object newValue = event.getRawNewValue(shouldApplyDelta());
+      final Object newValue = event.getRawNewValue();
       assert !(newValue instanceof StoredObject); // since we already called getOffHeapNewValue() and it returned null
       if (newValue instanceof CachedDeserializable) {
         this.value = ((CachedDeserializable) newValue).getSerializedValue();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
index 46ff263..e0f8b6f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
@@ -567,13 +567,8 @@ public class ParallelGatewaySenderQueue implements RegionQueue {
           if (isAccessor)
             return; // return from here if accessor node
 
-          // if the current node is marked uninitialized (SQLF DDL replay in
-          // progress) then we cannot wait for buckets to recover, because
-          // bucket creation has been disabled until DDL replay is complete.
-          if(!prQ.getCache().isUnInitializedMember(prQ.getDistributionManager().getId())) {
-            //Wait for buckets to be recovered.
-            prQ.shadowPRWaitForBucketRecovery();
-          }
+          //Wait for buckets to be recovered.
+          prQ.shadowPRWaitForBucketRecovery();
 
         } catch (IOException veryUnLikely) {
           logger.fatal(LocalizedMessage.create(LocalizedStrings.SingleWriteSingleReadRegionQueue_UNEXPECTED_EXCEPTION_DURING_INIT_OF_0,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/RegionAttributesCreation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/RegionAttributesCreation.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/RegionAttributesCreation.java
index 243d8c5..f79f7f9 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/RegionAttributesCreation.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/RegionAttributesCreation.java
@@ -203,12 +203,10 @@ public class RegionAttributesCreation extends UserSpecifiedRegionAttributes impl
     this(cc, getDefaultAttributes(cc), true);
   }
 
-  // used by sqlfabric
   public RegionAttributesCreation() {
     this(defaultAttributes, true);
   }
 
-  // used by sqlfabric
   public RegionAttributesCreation(RegionAttributes attrs, boolean defaults) {
     this(null, attrs, defaults);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/i18n/LocalizedStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/i18n/LocalizedStrings.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/i18n/LocalizedStrings.java
index 13eca56..a09952e 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/i18n/LocalizedStrings.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/i18n/LocalizedStrings.java
@@ -734,7 +734,6 @@ public class LocalizedStrings {
   public static final StringId PartitionedRegion_FAILED_REGISTRATION_PRID_0_NAMED_1 = new StringId(1872, "FAILED_REGISTRATION prId={0} named {1}");
   public static final StringId PartitionedRegion_FORCEREATTEMPT_EXCEPTION___0 = new StringId(1873, "ForceReattempt exception :  {0}");
   public static final StringId PartitionedRegion_NEWLY_ADDED_MEMBER_TO_THE_PR_IS_AN_ACCESSOR_AND_WILL_NOT_RECEIVE_INDEX_INFORMATION_0 = new StringId(1874, "Newly added member to the PR is an accessor and will not receive index information : {0}");
-  public static final StringId FunctionService_ROUTING_OBJECTS_SET_IS_NULL = new StringId(1875, "Set for routing objects passed is null");
 
   public static final StringId PartitionedRegion_NO_VM_AVAILABLE_FOR_CONTAINS_KEY_IN_1_ATTEMPTS = new StringId(1877, "No VM available for contains key in {1} attempts");
   public static final StringId PartitionedRegion_NO_VM_AVAILABLE_FOR_CONTAINS_VALUE_FOR_KEY_IN_1_ATTEMPTS = new StringId(1878, "No VM available for contains value for key in {1} attempts");
@@ -836,8 +835,6 @@ public class LocalizedStrings {
 
   public static final StringId LocalRegion_THE_FOLLOWING_EXCEPTION_OCCURRED_ATTEMPTING_TO_GET_KEY_0 = new StringId(1995, "The following exception occurred attempting to get key={0}");
 
-  public static final StringId RegionAdvisor_CANNOT_RESET_EXISTING_BUCKET = new StringId(
-                                                                                                1997, "For region {0}: RegionAdvisor cannot reset existing bucket with ID {1}");
   public static final StringId SystemAdmin_LRU_OPTION_HELP = new StringId(1998, "-lru=<type> Sets region''s lru algorithm. Valid types are: none, lru-entry-count, lru-heap-percentage, or lru-memory-size");
   public static final StringId SystemAdmin_LRUACTION_OPTION_HELP = new StringId(1999, "-lruAction=<action> Sets the region''s lru action. Valid actions are: none, overflow-to-disk, local-destroy");
   public static final StringId SystemAdmin_LRULIMIT_OPTION_HELP = new StringId(2000, "-lruLimit=<int> Sets the region''s lru limit. Valid values are >= 0");
@@ -2973,9 +2970,6 @@ public class LocalizedStrings {
   public static final StringId GemFireCache_INITIALIZATION_FAILED_FOR_REGION_0 = new StringId(4609, "Initialization failed for Region {0}");
   public static final StringId LocalRegion_INITIALIZATION_FAILED_FOR_REGION_0 = new StringId(4610, "Initialization failed for Region {0}");
   public static final StringId PartitionedRegion_0_EVICTIONATTRIBUTES_1_DO_NOT_MATCH_WITH_OTHER_2 = new StringId(4611, "For Partitioned Region {0} the locally configured EvictionAttributes {1} do not match with other EvictionAttributes {2} and may cause misses during reads from VMs with smaller maximums.");
-  public static final StringId DSFIDFactory_COULD_NOT_INSTANTIATE_SQLFABRIC_MESSAGE_CLASSID_0_1 = new StringId(4616, "Could not instantiate SQLFabric message [classId:{0}]:{1}");
-  public static final StringId DSFIDFactory_ILLEGAL_ACCESS_FOR_SQLFABRIC_MESSAGE_CLASSID_0_1 = new StringId(4617, "Illegal access for SQLFabric message [classId:{0}]:{1}");
-  public static final StringId DSFIDFactory_UNKNOWN_CLASSID_0_FOR_SQLFABRIC_MESSAGE = new StringId(4618, "Unknown ClassId [{0}] for SQLFabric message");
   public static final StringId AbstractRegionMap_THE_CURRENT_VALUE_WAS_NOT_EQUAL_TO_EXPECTED_VALUE = new StringId(4619, "The current value was not equal to expected value.");
   public static final StringId AbstractRegionEntry_THE_CURRENT_VALUE_WAS_NOT_EQUAL_TO_EXPECTED_VALUE = new StringId(4620, "The current value was not equal to expected value.");
   public static final StringId AbstractRegionMap_ENTRY_NOT_FOUND_WITH_EXPECTED_VALUE = new StringId(4621, "entry not found with expected value");
@@ -3112,11 +3106,6 @@ public class LocalizedStrings {
   public static final StringId AttributesFactory_INVALIDATE_REGION_NOT_SUPPORTED_FOR_PR = new StringId(4784,"ExpirationAction INVALIDATE or LOCAL_INVALIDATE for region is not supported for Partitioned Region.");
   public static final StringId AttributesFactory_LOCAL_DESTROY_IS_NOT_SUPPORTED_FOR_PR = new StringId(4785,"ExpirationAction LOCAL_DESTROY is not supported for Partitioned Region.");
   public static final StringId AttributesFactory_LOCAL_INVALIDATE_IS_NOT_SUPPORTED_FOR_PR = new StringId(4786,"ExpirationAction LOCAL_INVALIDATE is not supported for Partitioned Region.");
-  public static final StringId GemFireUtilLauncher_ARGUMENTS = new StringId(4789, "Usage:\n{0} [{1}] <arguments for the utility specified>\n\nThe command to display a particular utility''s usage is:\n{0} <utility name> --help");
-  public static final StringId GemFireUtilLauncher_INVALID_UTILITY_0 = new StringId(4790, "Invalid utility name: {0} was specified.");
-  public static final StringId GemFireUtilLauncher_PROBLEM_STARTING_0 = new StringId(4791, "Problem starting {0}.");
-  public static final StringId GemFireUtilLauncher_MISSING_COMMAND = new StringId(4792, "** Missing command");
-  public static final StringId GemFireUtilLauncher_HELP = new StringId(4793, "** Displaying help information");
   public static final StringId AttributesFactory_DESTROY_REGION_NOT_SUPPORTED_FOR_PR = new StringId(4794,"ExpirationAction DESTROY or LOCAL_DESTROY for region is not supported for Partitioned Region.");
   public static final StringId ExecuteFunction_DS_NOT_CREATED_OR_NOT_READY = new StringId(4795, "DistributedSystem is either not created or not ready");
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LoggingThreadGroup.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LoggingThreadGroup.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LoggingThreadGroup.java
index 82f487f..420b258 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LoggingThreadGroup.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LoggingThreadGroup.java
@@ -106,7 +106,6 @@ public class LoggingThreadGroup extends ThreadGroup {
       if (group == null) {
         group = new LoggingThreadGroup(name, logWriter);
         // force autoclean to false and not inherit from parent group
-        // (happens to be true for SQLFabric started threads as seen in #41438)
         group.setDaemon(false);
         loggingThreadGroups.add(group);
       }
@@ -161,7 +160,6 @@ public class LoggingThreadGroup extends ThreadGroup {
       if (group == null) {
         group = new LoggingThreadGroup(name, logger);
         // force autoclean to false and not inherit from parent group
-        // (happens to be true for SQLFabric started threads as seen in #41438)
         group.setDaemon(false);
         loggingThreadGroups.add(group);
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/ManagerLogWriter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/ManagerLogWriter.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/ManagerLogWriter.java
index 353813c..adc1a9e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/ManagerLogWriter.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/ManagerLogWriter.java
@@ -324,40 +324,6 @@ public class ManagerLogWriter extends LocalLogWriter  {
       }
     }
 
-    /* This method is only used by sqlfabric, use getLogNameForOldMainLog instead - xzhou
-     *  
-     */
-//    public static File getMainLogName(File log) {
-//      /*
-//       * this is just searching for the existing logfile name
-//       * we need to search for meta log file name
-//       *
-//       */
-//      File dir = log.getAbsoluteFile().getParentFile();
-//      int previousMainId = calcNextMainId(dir, true);
-//      // comment out the following to fix bug 31789
-////       if (previousMainId > 1) {
-////         previousMainId--;
-////       }
-//      previousMainId--;
-//      File result = null;
-//      do {
-//        previousMainId++;
-//        StringBuffer buf = new StringBuffer(log.getPath());
-//        int insertIdx = buf.lastIndexOf(".");
-//        if (insertIdx == -1) {
-//          buf
-//            .append(formatId(previousMainId))
-//            .append(formatId(1));
-//        } else {
-//          buf.insert(insertIdx, formatId(1));
-//          buf.insert(insertIdx, formatId(previousMainId));
-//        }
-//        result = new File(buf.toString());
-//      } while (result.exists());
-//      return result;
-//    }
-    
     /**
      * as a fix for bug #41474 we use "." if getParentFile returns null
      */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/PureLogWriter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/PureLogWriter.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/PureLogWriter.java
index 3279bf6..53bac4f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/PureLogWriter.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/PureLogWriter.java
@@ -186,7 +186,6 @@ public class PureLogWriter extends LogWriterImpl {
         return sw.toString();
     }
 
-    // split out header writing portion for SQLFabric logging
     protected void printHeader(PrintWriter pw, int msgLevel, Date msgDate,
         String connectionName, String threadName, long tid) {
       pw.println();
@@ -208,7 +207,6 @@ public class PureLogWriter extends LogWriterImpl {
       pw.print("] ");
     }
 
-    // made public for use by SQLFabric logging
     public String put(int msgLevel, Date msgDate, String connectionName,
         String threadName, long tid, String msg, String exceptionText) {
         String result = formatLogLine(msgLevel, msgDate, connectionName

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java
index f3f064a..cd5f7dd 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/OffHeapHelper.java
@@ -33,7 +33,6 @@ public class OffHeapHelper {
   
   /**
    * If o is off-heap then return its heap form; otherwise return o since it is already on the heap.
-   * Note even if o is sqlf off-heap byte[] or byte[][] the heap form will be created.
    */
   public static Object getHeapForm(Object o) {
     if (o instanceof StoredObject) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ReferenceCountHelperImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ReferenceCountHelperImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ReferenceCountHelperImpl.java
index 571c2d1..6745414 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ReferenceCountHelperImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/ReferenceCountHelperImpl.java
@@ -208,7 +208,6 @@ class ReferenceCountHelperImpl {
           for (int i=0; i < list.size(); i++) {
             RefCountChangeInfo info = list.get(i);
             if (owner instanceof RegionEntry) {
-              // use identity comparison on region entries since sqlf does some wierd stuff in the equals method
               if (owner == info.getOwner()) {
                 if (info.getUseCount() > 0) {
                   info.decUseCount();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/annotations/OffHeapIdentifier.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/annotations/OffHeapIdentifier.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/annotations/OffHeapIdentifier.java
index e275467..23e8dcf 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/annotations/OffHeapIdentifier.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/offheap/annotations/OffHeapIdentifier.java
@@ -32,13 +32,9 @@ public enum OffHeapIdentifier {
   TX_ENTRY_STATE("com.gemstone.gemfire.internal.cache.originalVersionId"),
   GATEWAY_SENDER_EVENT_IMPL_VALUE("com.gemstone.gemfire.internal.cache.wan.GatewaySenderEventImpl.valueObj"),
   TEST_OFF_HEAP_REGION_BASE_LISTENER("com.gemstone.gemfire.internal.offheap.OffHeapRegionBase.MyCacheListener.ohOldValue and ohNewValue"),
-  COMPACT_COMPOSITE_KEY_VALUE_BYTES("com.vmware.sqlfire.internal.engine.store.CompactCompositeKey.valueBytes"),
-  // TODO: HOOTS: Deal with this
   REGION_ENTRY_VALUE(""),
   ABSTRACT_REGION_ENTRY_PREPARE_VALUE_FOR_CACHE("com.gemstone.gemfire.internal.cache.AbstractRegionEntry.prepareValueForCache(...)"),
   ABSTRACT_REGION_ENTRY_FILL_IN_VALUE("com.gemstone.gemfire.internal.cache.AbstractRegionEntry.fillInValue(...)"),
-  COMPACT_EXEC_ROW_SOURCE("com.vmware.sqlfire.internal.engine.store.CompactExecRow.source"),
-  COMPACT_EXEC_ROW_WITH_LOBS_SOURCE("com.vmware.sqlfire.internal.engine.store.CompactExecRowWithLobs.source"),
   GEMFIRE_TRANSACTION_BYTE_SOURCE(""),
   
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/shared/NativeCalls.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/shared/NativeCalls.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/shared/NativeCalls.java
index ead57ce..c8e7361 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/shared/NativeCalls.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/shared/NativeCalls.java
@@ -17,7 +17,6 @@
 
 package com.gemstone.gemfire.internal.shared;
 
-import java.io.File;
 import java.io.FileDescriptor;
 import java.io.FileInputStream;
 import java.io.IOException;
@@ -38,21 +37,17 @@ import com.gemstone.gemfire.SystemFailure;
  * Encapsulates native C/C++ calls via JNA. To obtain an instance of
  * implementation for a platform, use {@link NativeCalls#getInstance()}.
  * 
- * This class is also referenced by ODBC/.NET drivers so it should not refer to
- * any classes other than standard JDK or those within the same package.
- * 
  * @since GemFire 8.0
  */
 public abstract class NativeCalls {
 
   /**
    * Static instance of NativeCalls implementation. This can be one of JNA
-   * implementations in <code>NativeCallsJNAImpl</code> or can fallback to a
+   * implementations in <code>NativeCallsJNAImpl</code> or can fall back to a
    * generic implementation in case JNA is not available for the platform.
    * 
-   * Note: this variable is deliberately not final since other drivers like
-   * those for ADO.NET or ODBC will plugin their own native implementations of
-   * NativeCalls.
+   * Note: this variable is deliberately not final so that other clients 
+   * can plug in their own native implementations of NativeCalls.
    */
   protected static NativeCalls instance;
 
@@ -60,7 +55,7 @@ public abstract class NativeCalls {
     NativeCalls inst;
     try {
       // try to load JNA implementation first
-      // we do it via reflection since some clients like ADO.NET/ODBC
+      // we do it via reflection since some clients
       // may not have it
       final Class<?> c = Class
           .forName("com.gemstone.gemfire.internal.shared.NativeCallsJNAImpl");
@@ -73,24 +68,8 @@ public abstract class NativeCalls {
       inst = null;
     }
     if (inst == null) {
-      // In case JNA implementations cannot be loaded, fallback to generic
-      // implementations.
-      // Other clients like ADO.NET/ODBC will plugin their own implementations.
-      try {
-        // using reflection to get the implementation based on OSProcess
-        // since this is also used by GemFireXD client; at some point all the
-        // functionality of OSProcess should be folded into the JNA impl
-        final Class<?> c = Class
-            .forName("com.gemstone.gemfire.internal.OSProcess$NativeOSCalls");
-        inst = (NativeCalls)c.newInstance();
-      } catch (VirtualMachineError e) {
-        SystemFailure.initiateFailure(e);
-        throw e;
-      } catch (Throwable t) {
-        SystemFailure.checkFailure();
-        // fallback to generic impl in case of a problem
-        inst = new NativeCallsGeneric();
-      }
+      // fall back to generic implementation in case of a problem
+      inst = new NativeCallsGeneric();
     }
     instance = inst;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/util/ArrayUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/util/ArrayUtils.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/util/ArrayUtils.java
index 4f15b17..04b1a15 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/util/ArrayUtils.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/util/ArrayUtils.java
@@ -255,7 +255,7 @@ public abstract class ArrayUtils {
         }
         else {
           first = false;
-          // for SQLFire show the first byte[] for byte[][] storage
+          // show the first byte[] for byte[][] storage
           objectStringWithBytes(o, sb);
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/util/concurrent/CustomEntryConcurrentHashMap.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/util/concurrent/CustomEntryConcurrentHashMap.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/util/concurrent/CustomEntryConcurrentHashMap.java
index 0000fb9..0842d71 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/util/concurrent/CustomEntryConcurrentHashMap.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/util/concurrent/CustomEntryConcurrentHashMap.java
@@ -24,7 +24,7 @@
  * implementations can be plugged in. These HashEntry objects are now assumed to
  * be immutable in the sense that they cannot and should not be cloned in a
  * rehash, and the rehash mechanism has been recoded using locking for that. For
- * GemFire/SQLFire, this is now used to plugin the RegionEntry implementation
+ * Geode, this is now used to plugin the RegionEntry implementation
  * directly as a HashEntry instead of having it as a value and then HashEntry as
  * a separate object having references to key/value which reduces the entry
  * overhead substantially. Other change is to add a "create" method that creates

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
index b5f4149..4ecf3e3 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
@@ -107,7 +107,6 @@ import com.gemstone.gemfire.internal.cache.TXManagerImpl;
 import com.gemstone.gemfire.internal.cache.TXStateProxy;
 import com.gemstone.gemfire.internal.cache.Token;
 import com.gemstone.gemfire.internal.cache.TombstoneService;
-import com.gemstone.gemfire.internal.cache.delta.Delta;
 import com.gemstone.gemfire.internal.cache.versions.RegionVersionHolder;
 import com.gemstone.gemfire.internal.cache.versions.RegionVersionVector;
 import com.gemstone.gemfire.internal.cache.versions.VMRegionVersionVector;
@@ -2964,145 +2963,6 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
   }
   
   /**
-   * Delta implementation for the delta tests, appends " 10" if it's a string,
-   * or adds 10 if it's an Integer
-   */
-  static class AddTen implements Delta, Serializable {
-
-    public Object apply(EntryEvent<?, ?> putEvent) {
-      Object oldValue = putEvent.getOldValue();
-      if (oldValue instanceof String) {
-        return (String)oldValue + " 10";
-      }
-      else if (oldValue instanceof Integer) {
-        return new Integer(((Integer)oldValue).intValue() + 10);
-      }
-      else throw new IllegalStateException("unexpected old value");
-    }
-
-    public Object merge(Object toMerge, boolean isCreate) {
-      return null;
-    }
-
-    public Object merge(Object toMerge) {
-      return null;
-    }
-
-    public Object getResultantValue() {
-      return null;
-    }
-  }
-
-  /**
-   * Tests that a Delta is applied correctly both locally and on a replicate
-   * region.
-   */
-  public void testDeltaWithReplicate() throws InterruptedException {
-    if (!supportsReplication()) {
-      return;
-    }
-    //pauseIfNecessary(100); // wait for previous tearDown to complete
-    
-    final String name = this.getUniqueName();
-    final Object key1 = "KEY1";
-    final Object value1 = "VALUE1";
-    final Object key2 = "KEY2";
-    final Object value2 = new Integer (0xCAFE);
-    final Object key3 = "KEY3";
-    final Object value3 = "VALUE3";
-    
-    final Delta delta = new AddTen();
-
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    VM vm2 = host.getVM(2); // use VM on separate shared memory in case shared regions
-    
-    SerializableRunnable create = new
-    CacheSerializableRunnable("Create Replicate Region") {
-      public void run2() throws CacheException {
-        RegionAttributes ra = getRegionAttributes();
-        AttributesFactory factory =
-          new AttributesFactory(ra);
-        if (ra.getEvictionAttributes() == null
-            || !ra.getEvictionAttributes().getAction().isOverflowToDisk()) {
-          factory.setDiskStoreName(null);
-        }
-        factory.setDataPolicy(DataPolicy.REPLICATE);
-        createRegion(name, factory.create());
-      }
-    };
-    
-    vm0.invoke(create);
-    Thread.sleep(250);
-    vm2.invoke(create);
-    Thread.sleep(250);
-    
-    vm0.invoke(new CacheSerializableRunnable("Put data") {
-      public void run2() throws CacheException {
-        Region region = getRootRegion().getSubregion(name);
-        region.put(key1, value1);
-        region.put(key2, value2);
-        region.put(key3, value3);
-      }
-    });
-    
-    Invoke.invokeRepeatingIfNecessary(vm2, new CacheSerializableRunnable("Wait for update") {
-      public void run2() throws CacheException {
-        Region region = getRootRegion().getSubregion(name);
-        assertNotNull(region.getEntry(key1));
-        assertNotNull(region.getEntry(key2));
-        assertNotNull(region.getEntry(key3));
-      }
-    }, getRepeatTimeoutMs());
-    
-    // apply delta
-    vm0.invoke(new CacheSerializableRunnable("Apply delta") {
-      public void run2() throws CacheException {
-        Region region = getRootRegion().getSubregion(name);
-        region.put(key1, delta);
-        region.put(key2, delta);
-        region.put(key3, delta);
-      }
-    });
-    
-    CacheSerializableRunnable verify = 
-      new CacheSerializableRunnable("Verify values") {
-        public void run2() throws CacheException {
-          Region region = getRootRegion().getSubregion(name);
-          
-          Region.Entry entry1 = region.getEntry(key1);
-          assertNotNull(entry1);
-          assertEquals("VALUE1 10", entry1.getValue());
-          
-          Region.Entry entry2 = region.getEntry(key2);
-          assertNotNull(entry2);
-          assertEquals(new Integer(0xCAFE + 10), entry2.getValue());
-          
-          Region.Entry entry3 = region.getEntry(key3);
-          assertNotNull(entry3);
-          assertEquals("VALUE3 10", entry3.getValue());
-        }
-      };
-    
-    Invoke.invokeRepeatingIfNecessary(vm0, verify, getRepeatTimeoutMs());
-    
-    
-    // Destroy the local entries so we know that they are not found by
-    // a netSearch
-    vm0.invoke(new CacheSerializableRunnable("Remove local entries") {
-      public void run2() throws CacheException {
-        Region region = getRootRegion().getSubregion(name);
-        region.localDestroyRegion();
-      }
-    });
-    
-    Invoke.invokeRepeatingIfNecessary(vm2, verify, getRepeatTimeoutMs());
-    
-  }
-  
-  
-  
-  /**
    * Tests that a newly-created mirrored region contains all of the
    * entries of another region.
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
index 3e153e0..3e133f6 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
@@ -115,7 +115,7 @@ public class LocatorDUnitTest extends DistributedTestCase {
   ////////  Test Methods
 
   /**
-   * SQLFire uses a colocated locator in a dm-type=normal VM.  This tests that
+   * This tests that
    * the locator can resume control as coordinator after all locators have been
    * shut down and one is restarted.  It's necessary to have a lock service
    * start so elder failover is forced to happen.  Prior to fixing how this worked

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
index 3538e41..06d0e08 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
@@ -36,11 +36,6 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 
-/**
- * Port of GemFireXD's corresponding test for distributed transactions
- * 
- *
- */
 @SuppressWarnings("deprecation")
 public class DistributedTransactionDUnitTest extends CacheTestCase {
   final protected String CUSTOMER_PR = "customerPRRegion";
@@ -345,11 +340,6 @@ public class DistributedTransactionDUnitTest extends CacheTestCase {
   
 
   
-  /**
-   * From GemFireXD: testTransactionalInsertOnReplicatedTable
-   * 
-   * @throws Exception
-   */
   public void testTransactionalPutOnReplicatedRegion() throws Exception {
     Host host = Host.getHost(0);
     VM server1 = host.getVM(0);
@@ -623,8 +613,6 @@ public class DistributedTransactionDUnitTest extends CacheTestCase {
   }
   
   /*
-   * [sjigyasu] This adapation of test from GemFireXD allows the difference in 
-   * the way GemFire and GemFireXD handle server groups.
    * We create 2 partitioned regions one on each server and have a third node
    * as accessor and fire transactional operations on it.
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java
index 07f87a3..ca870cc 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java
@@ -167,15 +167,6 @@ public class BackwardCompatibilitySerializationDUnitTest extends CacheTestCase {
     constdsfids.add(new Byte(DataSerializableFixedID.RESULTS_BAG).intValue());
     constdsfids.add(new Byte(DataSerializableFixedID.GATEWAY_EVENT_IMPL_66)
         .intValue());
-    constdsfids.add(new Byte(DataSerializableFixedID.SQLF_TYPE).intValue());
-    constdsfids.add(new Byte(DataSerializableFixedID.SQLF_DVD_OBJECT)
-        .intValue());
-    constdsfids.add(new Byte(DataSerializableFixedID.SQLF_GLOBAL_ROWLOC)
-        .intValue());
-    constdsfids.add(new Byte(DataSerializableFixedID.SQLF_GEMFIRE_KEY)
-        .intValue());
-    constdsfids.add(new Byte(DataSerializableFixedID.SQLF_FORMATIBLEBITSET)
-        .intValue());
     constdsfids
         .add(new Short(DataSerializableFixedID.TOKEN_INVALID).intValue());
     constdsfids.add(new Short(DataSerializableFixedID.TOKEN_LOCAL_INVALID)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRCustomPartitioningDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRCustomPartitioningDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRCustomPartitioningDUnitTest.java
index 9462fab..3e83159 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRCustomPartitioningDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/PRCustomPartitioningDUnitTest.java
@@ -465,8 +465,7 @@ public class PRCustomPartitioningDUnitTest extends
 
 /**
  * Example implementation of a Partition Resolver which uses part of the value
- * for custom partitioning.  This example is a simplification of what SQLFabric
- * may do when the DDL specifies "partition by"    
+ * for custom partitioning.
 
  */
 class MonthBasedPartitionResolver implements PartitionResolver, Declarable2 {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java
index e08a268..bc974f7 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessageJUnitTest.java
@@ -73,7 +73,6 @@ public class FetchEntriesMessageJUnitTest {
     cache = Fakes.cache();
     PartitionedRegion pr = mock(PartitionedRegion.class);
     InternalDistributedSystem system = cache.getDistributedSystem();
-    when(pr.keyRequiresRegionContext()).thenReturn(false);
 
     FetchEntriesResponse response = new FetchEntriesResponse(system, pr, null, 0);
     HeapDataOutputStream chunkStream = createDummyChunk();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt b/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
index cd659e6..37eee1b 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedDataSerializables.txt
@@ -963,7 +963,7 @@ fromData,284,2bb9002301003d1c077e07a0000704a70004033e1c10207e1020a0000704a700040
 toData,299,033d033e2ab600089e003d1c07803d043e2ab6000b3a041904b9000c01009900271904b9000d0100c0000a3a051905c600131905c1001199000e1c1020803da70006a7ffd5b80012b20013b90014020099001fb80012b20013121505bd001659032a5359041cb8001753b9001804002b1cb9001902001d9900b32ab60008852bb8001abb001b592ab60008b7001c3a040336052ab6000b3a061906b9000c010099008a1906b9000d0100c0000a3a071907c7000d2b03b900190200a7006c1907b6001d3a081908c700132b04b90019020019072bb8001ea7005019041908b6001f360915099a00242b05b90019020084050115053609190419081509b600205719072bb8001ea700212b06b90019020019072b03b6002119071908b6002215090464852bb8001aa7ff72b1
 
 com/gemstone/gemfire/internal/cache/DistributedPutAllOperation$PutAllEntryData,1
-toData,236,2ab4000a4e2ab4000c3a042d2bb8003d1904c1003e9a00081904c700192b03b9003f02001904c0003ec0003e2bb80040a700341904c1004199001f1904c000413a052b04b9003f02001905b9004201002bb80040a700102b04b9003f020019042bb800432b2ab40012b40044b9003f02002ab4000636052ab40026c6000a150507809136052ab40017c6001d15051008809136052ab40017c1004599000b150510208091360515051080809136052b1505b9003f02002ab40026c6000b2ab400262bb8003d2ab40017c6000b2ab400172bb800462ab6002899000b2ab400142bb800462ab400082bb80047b1
+toData,229,2ab4000a4d2ab4000c4e2c2bb8003d2dc1003e9a00072dc700182b03b9003f02002dc0003ec0003e2bb80040a700312dc1004199001e2dc000413a042b04b9003f02001904b9004201002bb80040a7000f2b04b9003f02002d2bb800432b2ab40012b40044b9003f02002ab4000636042ab40026c6000a150407809136042ab40017c6001d15041008809136042ab40017c1004599000b150410208091360415041080809136042b1504b9003f02002ab40026c6000b2ab400262bb8003d2ab40017c6000b2ab400172bb800462ab6002899000b2ab400142bb800462ab400082bb80047b1
 
 com/gemstone/gemfire/internal/cache/DistributedPutAllOperation$PutAllMessage,2
 fromData,197,2a2bb7003c2a2bb8003dc0003eb500052a2bb8003f88b500152a2ab40015bd0040b500062ab400159e00722bb800414dbb004259b700434e03360415042ab40015a200202ab400061504bb0040592b2ab4000515042c2db7004453840401a7ffdd2bb9004501003604150499002f2bb800463a0503360615062ab40015a2001d2ab4000615063219051506b60047c00048b5002e840601a7ffe02ab400491140007e99000e2a2bb8003dc0004bb5000b2a2ab400491180007e99000704a7000403b5001ab1
@@ -973,8 +973,9 @@ com/gemstone/gemfire/internal/cache/DistributedRegionFunctionStreamingMessage,2
 fromData,171,2a2bb700612bb9006201003d1c047e9900142a2bb900640100b500082ab40008b800651c077e99000d2a2bb900640100b500051c057e99000e2a2bb80066c00067b500062bb800664e2dc100689900252a03b5000d2a2dc00068b80069b500072ab40007c7001b2a2dc00068b5004ca700102a2dc0006ab500072a04b5000d2a2bb80066c0006bb500092a2bb8006cb5000b2a2bb8006db5000a2a1c10407e99000704a7000403b5000cb1
 toData,173,2a2bb7006f033d2ab400089900081c0480933d2ab40005029f00081c0780933d2ab40006c600081c0580933d2ab4000c9900091c104080933d2b1cb9007002002ab4000899000d2b2ab40008b9007102002ab40005029f000d2b2ab40005b9007102002ab40006c6000b2ab400062bb800722ab4000d99000e2ab400072bb80072a700102ab40007b9005701002bb800722ab400092bb800722ab4000bc000732bb800742ab4000a2bb80075b1
 
+
 com/gemstone/gemfire/internal/cache/DistributedRemoveAllOperation$RemoveAllEntryData,1
-toData,146,2ab4000a4e2d2bb8003f2b2ab40010b40040b9004102002ab4000636042ab40022c6000a150407809136042ab40015c6001d15041008809136042ab40015c1004299000b150410208091360415041080809136042b1504b9004102002ab40022c6000b2ab400222bb8003f2ab40015c6000b2ab400152bb800432ab6002499000b2ab400122bb800432ab400082bb80044b1
+toData,136,2ab4000a4d2c2bb8003f2b2ab40010b40040b9004102002ab400063e2ab40022c600081d0780913e2ab40015c600191d100880913e2ab40015c100429900091d102080913e1d108080913e2b1db9004102002ab40022c6000b2ab400222bb8003f2ab40015c6000b2ab400152bb800432ab6002499000b2ab400122bb800432ab400082bb80044b1
 
 com/gemstone/gemfire/internal/cache/DistributedRemoveAllOperation$RemoveAllMessage,2
 fromData,197,2a2bb700382a2bb80039c0003ab500032a2bb8003b88b500132a2ab40013bd003cb500042ab400139e00722bb8003d4dbb003e59b7003f4e03360415042ab40013a200202ab400041504bb003c592b2ab4000315042c2db7004053840401a7ffdd2bb9004101003604150499002f2bb800423a0503360615062ab40013a2001d2ab4000415063219051506b60043c00044b5002b840601a7ffe02ab400451140007e99000e2a2bb80039c00047b500092a2ab400451180007e99000704a7000403b50018b1
@@ -1075,8 +1076,8 @@ fromData,16,2a2bb7000d2a2bb9000e0100b50002b1
 toData,16,2a2bb7000f2b2ab40002b900100200b1
 
 com/gemstone/gemfire/internal/cache/InitialImageOperation$Entry,2
-fromData,107,2a2bb9001b0100b500032bb9001b01003d2a2bb8001cb500122ab40003b8001499000d2ab2001db50002a7001d2ab600159a000e2a2bb8001eb50002a7000b2a2bb8001fb500022a2bb900200100b500041c047e9900162a1c057e99000704a70004032bb80021b5000db1
-toData,125,2b2ab40003b9000f02002ab4000dc6000704a70004033d1c2ab4000dc1001199000705a700040380913d2b1cb9000f02002ab400122bb800132ab40003b800149a00232ab600159a000e2ab400022bb80016a700112ab40002c00017c000172bb800182b2ab40004b9001903002ab4000dc6000b2ab4000d2bb8001ab1
+fromData,89,2a2bb900150100b500032bb9001501003d2a2bb80016b5000f2ab40003b8001199000d2ab20017b50002a7000b2a2bb80018b500022a2bb900190100b500041c047e9900162a1c057e99000704a70004032bb8001ab5000ab1
+toData,101,2b2ab40003b9000c02002ab4000ac6000704a70004033d1c2ab4000ac1000e99000705a700040380913d2b1cb9000c02002ab4000f2bb800102ab40003b800119a000b2ab400022bb800122b2ab40004b9001303002ab4000ac6000b2ab4000a2bb80014b1
 
 com/gemstone/gemfire/internal/cache/InitialImageOperation$FilterInfoMessage,2
 fromData,230,2a2bb7008c2a2bb8008db500202ab4000403322bb8008db5003d2ab4000403322bb8008db500402ab4000403322bb8008db500422ab4000403322bb8008db500442ab4000403322bb8008db500462ab4000403322bb8008db500482ab4000403322bb8008db5004a2ab4000403322bb8008db5004c2ab4000404322bb8008db5003d2ab4000404322bb8008db500402ab4000404322bb8008db500422ab4000404322bb8008db500442ab4000404322bb8008db500462ab4000404322bb8008db500482ab4000404322bb8008db5004a2ab4000404322bb8008db5004c2a2bb8008db50033b1
@@ -1175,7 +1176,7 @@ fromData,9,2a2bb8000eb50002b1
 toData,9,2ab400022bb8000fb1
 
 com/gemstone/gemfire/internal/cache/QueuedOperation,1
-toData,97,2b2ab40002b40035b9003602002ab400072bb800372ab40002b600319900442ab400032bb800372ab40002b600169a000d2ab40002b600159900282b2ab40006b9003602002ab40006049f000e2ab400042bb80038a7000b2ab400052bb80037b1
+toData,78,2b2ab40002b40035b9003602002ab400072bb800372ab40002b600319900312ab400032bb800372ab40002b600169a000d2ab40002b600159900152b2ab40006b9003602002ab400042bb80038b1
 
 com/gemstone/gemfire/internal/cache/RegionEventImpl,2
 fromData,48,2a2bb80025b500092a2bb80026b500022a2bb900270100b80028b5000a2a2bb900290100b500032a2bb8002ab5000bb1
@@ -1370,8 +1371,8 @@ fromData,37,2a2bb7004f2a2bb900500100b500052a2bb900500100b500032a2bb80051c00052b5
 toData,34,2a2bb7004c2b2ab40005b9004d02002b2ab40003b9004d02002ab400022bb8004eb1
 
 com/gemstone/gemfire/internal/cache/TXRegionLockRequestImpl,2
-fromData,102,2a2bb8000cb500032a03b7000d4d2bb8000e3e0336042cc600311d9e002d2a2c2ab40003b6000fc00010b500022ab40002c600172a2a2ab40002b600111d2bb70012b5000404360415049a00121d9e000e2a2a031d2bb70012b50004a700094e2a01b50004b1
-toData,17,2ab600212bb800222ab400042bb80023b1
+fromData,62,2a2bb8000cb500032a03b7000d4d2bb8000e3e2cc600161d9e00122a2c2ab40003b6000fc00010b500022a2a1d2bb70011b50004a700094e2a01b50004b1
+toData,17,2ab6001e2bb8001f2ab400042bb80020b1
 
 com/gemstone/gemfire/internal/cache/TXRemoteCommitMessage$TXRemoteCommitReplyMessage,2
 fromData,17,2a2bb7001a2a2bb8001bc0001cb50004b1
@@ -1410,16 +1411,16 @@ fromData,62,2a2bb700152bb9001601003d1c02a0000b2a01b50007a700271cbd00174e03360415
 toData,59,2a2bb700192ab40007c7000d2b02b9001a0200a700272ab40007be3d2b1cb9001a0200033e1d1ca200132ab400071d322bb8001b840301a7ffeeb1
 
 com/gemstone/gemfire/internal/cache/UpdateAttributesProcessor$UpdateAttributesMessage,2
-fromData,71,2a2bb7002a2a2bb8002bb500082a2bb9002c0100b500022ab40002b8002d2a2bb8002ec0000eb50005a700094d2a01b500052a2bb900300100b500032a2bb900300100b50004b1
-toData,52,2a2bb700312ab400082bb800322b2ab40002b9003302002ab400052bb800342b2ab40003b9003502002b2ab40004b900350200b1
+fromData,62,2a2bb7002a2a2bb8002bb500082a2bb9002c0100b500022ab40002b8002d2a2bb8002ec0000eb500052a2bb9002f0100b500032a2bb9002f0100b50004b1
+toData,52,2a2bb700302ab400082bb800312b2ab40002b9003202002ab400052bb800332b2ab40003b9003402002b2ab40004b900340200b1
 
 com/gemstone/gemfire/internal/cache/UpdateEntryVersionOperation$UpdateEntryVersionMessage,2
 fromData,45,2a2bb700322a2bb80033c00034b500022a2bb80033b500092bb800354d2cb6003699000b2a2bb80037b50005b1
 toData,118,2a2bb700382ab400022bb800392ab400092bb800392ab40003b6003ac0001e4d2cc1003b99002e2cb6003c4e2db6003d990018b2003e2bb8003f2ab40003b600402bb80041a7000ab200422bb8003fa700262cb60043990018b2003e2bb8003f2ab40003b600402bb80041a7000ab200422bb8003fb1
 
 com/gemstone/gemfire/internal/cache/UpdateOperation$UpdateMessage,2
-fromData,163,2a2bb7006c2bb9006d01003d1cb2006e7e99000704a70004033e1d9900332abb006f59b70070b500042ab400042bb800712bb900720100360415049900102a2bb900730100b80006b50007a700082a01b500042a2bb80074b5000d2a1cb200757e91b5000a2ab6002899000e2a2bb80076b50026a7002e2ab4000a04a0000e2a2bb80074b50010a7000b2a2bb80076b5000f1cb200777e99000b2a2bb80076b50026b1
-toData,235,2ab40003b60039c000784d2a2cb700792a2bb7007a2ab4000a3e2ab40004c6000a1db2006e80913e2ab4000a99001b2ab400059900142ab40003b6005bc6000a1db2007780913e2b1db9007b02002ab40004c6004b2ab400042bb8007c2cc1007d9900352cb6007e3a041904b6007f9a000d2b03b900800200a7001a2b04b9008002002b2ab40003b60081b60082b900830300a7000a2b03b9008002002ab4000d2bb800842ab6002899001e2ab40003b6005b2bb800852ab40003b60039b60086b60087a700262ab4000a2ab400102ab4000f2bb800881db200777e99000e2ab40003b6005b2bb80085b1
+fromData,144,2a2bb700692bb9006a01003d1cb2006b7e99000704a70004033e1d9900332abb006c59b7006db500042ab400042bb8006e2bb9006f0100360415049900102a2bb900700100b80006b50007a700082a01b500042a2bb80071b5000d2a1cb200727e91b5000a2ab6002899000e2a2bb80073b50026a7001b2a2bb80073b5000f1cb200747e99000b2a2bb80073b50026b1
+toData,235,2ab40003b60039c000754d2a2cb700762a2bb700772ab4000a3e2ab40004c6000a1db2006b80913e2ab4000a99001b2ab400059900142ab40003b60058c6000a1db2007480913e2b1db9007802002ab40004c6004b2ab400042bb800792cc1007a9900352cb6007b3a041904b6007c9a000d2b03b9007d0200a7001a2b04b9007d02002b2ab40003b6007eb6007fb900800300a7000a2b03b9007d02002ab4000d2bb800812ab6002899001e2ab40003b600582bb800822ab40003b60039b60083b60084a700262ab4000a2ab400102ab4000f2bb800851db200747e99000e2ab40003b600582bb80082b1
 
 com/gemstone/gemfire/internal/cache/UpdateOperation$UpdateWithContextMessage,2
 fromData,14,2a2bb700132a2bb80014b50009b1
@@ -1430,8 +1431,8 @@ fromData,17,2bb800204d2a2cbeb500082a2cb50006b1
 toData,9,2ab600212bb80022b1
 
 com/gemstone/gemfire/internal/cache/WrappedCallbackArgument,2
-fromData,9,2a2bb80005b50003b1
-toData,24,2ab4000299000e2ab400032bb80004a70008012bb80004b1
+fromData,9,2a2bb80004b50002b1
+toData,9,2ab400022bb80003b1
 
 com/gemstone/gemfire/internal/cache/compression/CompressedCachedDeserializable,2
 fromData,18,2a2ab600072bb8000eb900080200b50002b1

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt b/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
index 4e951af..8ea91f5 100755
--- a/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/codeAnalysis/sanctionedSerializables.txt
@@ -245,7 +245,6 @@ com/gemstone/gemfire/internal/AbstractConfig$SortedProperties,true,7156507110684
 com/gemstone/gemfire/internal/ConfigSource,true,-4097017272431018553,description:java/lang/String,type:com/gemstone/gemfire/internal/ConfigSource$Type
 com/gemstone/gemfire/internal/ConfigSource$Type,false
 com/gemstone/gemfire/internal/CopyOnWriteHashSet,true,8591978652141659932
-com/gemstone/gemfire/internal/DSFIDFactory$SqlfSerializationException,true,5076687296705595933
 com/gemstone/gemfire/internal/DSFIDNotFoundException,true,130596009484324655,dsfid:int,versionOrdinal:short
 com/gemstone/gemfire/internal/InternalDataSerializer$SERIALIZATION_VERSION,false
 com/gemstone/gemfire/internal/InternalStatisticsDisabledException,true,4146181546364258311


[5/5] incubator-geode git commit: GEODE-1464: remove sqlf code

Posted by ds...@apache.org.
GEODE-1464: remove sqlf code


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/880f8648
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/880f8648
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/880f8648

Branch: refs/heads/develop
Commit: 880f86483f6b75775f34e6821046ba18deb933f0
Parents: 6967ac1
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Mon Jun 6 18:01:18 2016 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Mon Jun 6 18:01:18 2016 -0700

----------------------------------------------------------------------
 .../cache/query/internal/IndexUpdater.java      | 123 --------
 .../internal/DistributionMessage.java           |   2 +-
 .../internal/InternalDistributedSystem.java     |  43 +--
 .../distributed/internal/ReplyProcessor21.java  |   6 +-
 .../com/gemstone/gemfire/internal/DSCODE.java   |  16 +-
 .../gemstone/gemfire/internal/DSFIDFactory.java | 187 -------------
 .../internal/DataSerializableFixedID.java       |  56 +---
 .../gemfire/internal/GemFireUtilLauncher.java   | 165 -----------
 .../internal/InternalDataSerializer.java        |   7 -
 .../gemstone/gemfire/internal/SystemAdmin.java  |   2 -
 .../com/gemstone/gemfire/internal/Version.java  |  21 +-
 .../gemfire/internal/VersionedDataStream.java   |   5 +-
 .../internal/cache/AbstractDiskRegionEntry.java |   5 -
 .../internal/cache/AbstractRegionEntry.java     |  30 +-
 .../internal/cache/AbstractRegionMap.java       | 238 ++--------------
 .../gemfire/internal/cache/BucketAdvisor.java   |   6 -
 .../gemfire/internal/cache/BucketRegion.java    |  45 +--
 .../cache/CacheDistributionAdvisor.java         |  28 +-
 .../internal/cache/CacheServerLauncher.java     |   2 +-
 .../cache/CachedDeserializableFactory.java      |   7 -
 .../internal/cache/ColocationHelper.java        |  67 +----
 .../internal/cache/DestroyOperation.java        |   3 -
 .../gemfire/internal/cache/DiskEntry.java       | 152 ++--------
 .../gemfire/internal/cache/DiskInitFile.java    |   2 -
 .../gemfire/internal/cache/DiskRegion.java      |   2 -
 .../internal/cache/DiskWriteAttributesImpl.java |  91 ------
 .../gemfire/internal/cache/DistTXState.java     |   4 +-
 .../cache/DistTXStateOnCoordinator.java         |   2 +-
 .../cache/DistributedCacheOperation.java        |  31 +-
 .../cache/DistributedPutAllOperation.java       |  52 +---
 .../internal/cache/DistributedRegion.java       |  37 +--
 ...stributedRegionFunctionStreamingMessage.java |   2 -
 .../cache/DistributedRemoveAllOperation.java    |  24 +-
 .../gemfire/internal/cache/EntryBits.java       |  22 --
 .../gemfire/internal/cache/EntryEventImpl.java  | 255 +----------------
 .../internal/cache/EntryOperationImpl.java      |  15 +-
 .../internal/cache/GemFireCacheImpl.java        | 159 +++++------
 .../gemfire/internal/cache/GridAdvisor.java     |   6 +-
 .../internal/cache/InitialImageOperation.java   |  96 +------
 .../internal/cache/InternalRegionArguments.java |  22 --
 .../internal/cache/InvalidateOperation.java     |   3 -
 .../gemfire/internal/cache/KeyInfo.java         |   5 +-
 .../internal/cache/KeyWithRegionContext.java    |  70 -----
 .../gemfire/internal/cache/ListOfDeltas.java    | 100 -------
 .../gemfire/internal/cache/LocalRegion.java     | 131 +--------
 .../gemstone/gemfire/internal/cache/Oplog.java  |  24 +-
 .../internal/cache/PRHARedundancyProvider.java  |   8 -
 .../internal/cache/PartitionAttributesImpl.java |  10 -
 .../internal/cache/PartitionedRegion.java       | 280 +------------------
 .../cache/PartitionedRegionDataStore.java       |  16 --
 .../gemfire/internal/cache/ProxyRegionMap.java  |  23 +-
 .../gemfire/internal/cache/QueuedOperation.java |  16 +-
 .../gemfire/internal/cache/RegionEntry.java     |   6 +-
 .../gemfire/internal/cache/RegionMap.java       |   8 +-
 .../cache/RemoteContainsKeyValueMessage.java    |   3 -
 .../internal/cache/RemoteDestroyMessage.java    |  13 +-
 .../internal/cache/RemoteFetchEntryMessage.java |   3 -
 .../cache/RemoteFetchVersionMessage.java        |   3 -
 .../internal/cache/RemoteGetMessage.java        |   3 -
 .../internal/cache/RemoteInvalidateMessage.java |   3 -
 .../internal/cache/RemotePutAllMessage.java     |  15 +-
 .../internal/cache/RemotePutMessage.java        |  32 +--
 .../internal/cache/RemoteRemoveAllMessage.java  |   6 +-
 .../cache/SearchLoadAndWriteProcessor.java      |   3 -
 .../gemfire/internal/cache/TXEntry.java         |  11 -
 .../gemfire/internal/cache/TXEntryState.java    |  32 +--
 .../internal/cache/TXRegionLockRequestImpl.java |  14 +-
 .../gemfire/internal/cache/TXRegionState.java   |   5 -
 .../gemfire/internal/cache/TXState.java         |   5 +-
 .../internal/cache/TXStateInterface.java        |   1 -
 .../cache/UpdateAttributesProcessor.java        |   9 +-
 .../cache/UpdateEntryVersionOperation.java      |   5 -
 .../gemfire/internal/cache/UpdateOperation.java |  36 +--
 .../internal/cache/ValidatingDiskRegion.java    |   3 -
 .../internal/cache/WrappedCallbackArgument.java |  26 +-
 .../gemfire/internal/cache/delta/Delta.java     |  56 ----
 .../cache/execute/AbstractExecution.java        |  18 --
 .../FunctionStreamingResultCollector.java       |   3 +-
 .../cache/execute/InternalExecution.java        |  27 +-
 .../cache/execute/InternalFunctionService.java  |   2 +-
 .../execute/InternalRegionFunctionContext.java  |   5 -
 .../cache/execute/MemberFunctionExecutor.java   |   7 -
 .../execute/MultiRegionFunctionExecutor.java    |   7 -
 .../execute/RegionFunctionContextImpl.java      |  13 -
 .../cache/execute/ServerFunctionExecutor.java   |   7 -
 .../partitioned/ContainsKeyValueMessage.java    |   4 -
 .../cache/partitioned/DestroyMessage.java       |   4 -
 .../partitioned/FetchBulkEntriesMessage.java    |   6 -
 .../cache/partitioned/FetchEntriesMessage.java  |   6 -
 .../cache/partitioned/FetchEntryMessage.java    |   4 -
 .../cache/partitioned/FetchKeysMessage.java     |   6 -
 .../internal/cache/partitioned/GetMessage.java  |  16 +-
 .../cache/partitioned/InvalidateMessage.java    |   4 -
 .../cache/partitioned/PREntriesIterator.java    |   8 +-
 .../PRUpdateEntryVersionMessage.java            |   5 -
 .../cache/partitioned/PartitionMessage.java     |  18 --
 .../cache/partitioned/PutAllPRMessage.java      |  26 +-
 .../internal/cache/partitioned/PutMessage.java  |  36 +--
 .../cache/partitioned/RegionAdvisor.java        |  17 --
 .../partitioned/RemoteFetchKeysMessage.java     |   6 -
 .../cache/partitioned/RemoveAllPRMessage.java   |  11 +-
 .../rebalance/PartitionedRegionLoadModel.java   |  19 +-
 .../sockets/command/GatewayReceiverCommand.java |  13 -
 .../internal/cache/tx/DistTxEntryEvent.java     |  14 +-
 .../cache/wan/AbstractGatewaySender.java        |   2 +-
 .../AbstractGatewaySenderEventProcessor.java    |   2 +-
 .../wan/GatewaySenderEventCallbackArgument.java |   8 +-
 .../cache/wan/GatewaySenderEventImpl.java       |  13 +-
 .../parallel/ParallelGatewaySenderQueue.java    |   9 +-
 .../xmlcache/RegionAttributesCreation.java      |   2 -
 .../gemfire/internal/i18n/LocalizedStrings.java |  11 -
 .../internal/logging/LoggingThreadGroup.java    |   2 -
 .../internal/logging/ManagerLogWriter.java      |  34 ---
 .../gemfire/internal/logging/PureLogWriter.java |   2 -
 .../gemfire/internal/offheap/OffHeapHelper.java |   1 -
 .../offheap/ReferenceCountHelperImpl.java       |   1 -
 .../offheap/annotations/OffHeapIdentifier.java  |   4 -
 .../gemfire/internal/shared/NativeCalls.java    |  33 +--
 .../gemfire/internal/util/ArrayUtils.java       |   2 +-
 .../CustomEntryConcurrentHashMap.java           |   2 +-
 .../gemfire/cache30/MultiVMRegionTestCase.java  | 140 ----------
 .../gemfire/distributed/LocatorDUnitTest.java   |   2 +-
 .../disttx/DistributedTransactionDUnitTest.java |  12 -
 ...wardCompatibilitySerializationDUnitTest.java |   9 -
 .../execute/PRCustomPartitioningDUnitTest.java  |   3 +-
 .../FetchEntriesMessageJUnitTest.java           |   1 -
 .../sanctionedDataSerializables.txt             |  27 +-
 .../codeAnalysis/sanctionedSerializables.txt    |   1 -
 128 files changed, 292 insertions(+), 3368 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/IndexUpdater.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/IndexUpdater.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/IndexUpdater.java
deleted file mode 100644
index facbdf2..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/query/internal/IndexUpdater.java
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.cache.query.internal;
-
-import com.gemstone.gemfire.cache.EntryEvent;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.TimeoutException;
-import com.gemstone.gemfire.distributed.LockNotHeldException;
-import com.gemstone.gemfire.internal.cache.BucketRegion;
-import com.gemstone.gemfire.internal.cache.EntryEventImpl;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.RegionEntry;
-
-public interface IndexUpdater {
-
-  /**
-   * This method is invoked when an entry is added, updated or destroyed in a
-   * region for index maintenance. This method will do some pre-update
-   * operations for the index like constraint checks or any other logging that
-   * may be required, and any index updates if required.
-   * 
-   * @param owner
-   *          the {@link Region} that owns this event; will be different from
-   *          {@link EntryEvent#getRegion()} for partitioned regions
-   * @param event
-   *          the {@link EntryEvent} representing the operation.
-   * @param entry
-   *          the region entry.
-   */
-  public void onEvent(LocalRegion owner, EntryEventImpl event, RegionEntry entry);
-
-  /**
-   * This method is invoked after an entry has been added, updated or destroyed
-   * in a region for index maintenance. This method will commit the changes to
-   * the indexes or may rollback some of the changes done in {@link #onEvent} if
-   * the entry operation failed for some reason.
-   * 
-   * @param owner
-   *          the {@link Region} that owns this event; will be different from
-   *          {@link EntryEvent#getRegion()} for partitioned regions
-   * @param event
-   *          the {@link EntryEvent} representing the operation.
-   * @param entry
-   *          the region entry.
-   * @param success
-   *          true if the entry operation succeeded and false otherwise.
-   */
-  public void postEvent(LocalRegion owner, EntryEventImpl event,
-      RegionEntry entry, boolean success);
-
-  /**
-   * Invoked to clear all index entries for a bucket before destroying it.
-   * 
-   * @param baseBucket
-   *          the {@link BucketRegion} being destroyed
-   * @param bucketId
-   *          the ID of the bucket being destroyed
-   */
-  public void clearIndexes(BucketRegion baseBucket, int bucketId);
-
-  /**
-   * Take a read lock indicating that bucket/region GII is in progress to block
-   * index list updates during the process.
-   * 
-   * This is required to be a reentrant lock. The corresponding write lock that
-   * will be taken by the implementation internally should also be reentrant.
-   * 
-   * @throws TimeoutException
-   *           in case of timeout in acquiring the lock
-   */
-  public void lockForGII() throws TimeoutException;
-
-  /**
-   * Release the read lock taken for GII by {@link #lockForGII()}.
-   * 
-   * @throws LockNotHeldException
-   *           if the current thread does not hold the read lock for GII
-   */
-  public void unlockForGII() throws LockNotHeldException;
-
-  /**
-   * Take a read lock to wait for completion of any index load in progress
-   * during initial DDL replay. This is required since no table level locks are
-   * acquired during initial DDL replay to avoid blocking most (if not all) DMLs
-   * in the system whenever a new node comes up.
-   * 
-   * This will be removed at some point when we allow for concurrent loading and
-   * initialization of index even while operations are in progress using
-   * something similar to region GII token mode for indexes or equivalent (bug
-   * 40899).
-   * 
-   * This is required to be a reentrant lock. The corresponding write lock that
-   * will be taken by the implementation internally should also be reentrant.
-   * 
-   * @return true if locking was required and was acquired and false if it was
-   *         not required
-   * @throws TimeoutException
-   *           in case of timeout in acquiring the lock
-   */
-  public boolean lockForIndexGII() throws TimeoutException;
-
-  /**
-   * Release the read lock taken for GII by {@link #lockForIndexGII()}.
-   * 
-   * @throws LockNotHeldException
-   *           if the current thread does not hold the read lock
-   */
-  public void unlockForIndexGII() throws LockNotHeldException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionMessage.java
index 3a64d06..85a4269 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionMessage.java
@@ -192,7 +192,7 @@ public abstract class DistributionMessage
         return true;
       case DistributionManager.REGION_FUNCTION_EXECUTION_EXECUTOR:
         // allow nested distributed functions to be executed from within the
-        // execution of a function; this is required particularly for SQLFabric
+        // execution of a function
         // TODO: this can later be adjusted to use a separate property
         return false;
       default:

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
index 9d49b49..af81cc1 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/InternalDistributedSystem.java
@@ -184,9 +184,6 @@ public class InternalDistributedSystem
    * a live locator can be contacted.
    */
   private QuorumChecker quorumChecker;
-  
-  /** sqlfire disconnect listener */
-  private DisconnectListener sqlfDisconnectListener;
 
   /**
    * A Constant that matches the ThreadGroup name of the shutdown hook.
@@ -2115,40 +2112,6 @@ public class InternalDistributedSystem
       }
     }
   }
-  
-  /**
-   * sqlfire's disconnect listener is invoked before the cache is closed when
-   * there is a forced disconnect
-   */
-  public void setSqlfForcedDisconnectListener(DisconnectListener listener) {
-    synchronized(this.listeners) { 
-      this.sqlfDisconnectListener = listener;
-    }
-  }
-  
-  private void notifySqlfForcedDisconnectListener() {
-    if (this.sqlfDisconnectListener != null) {
-      final boolean isDebugEnabled = logger.isDebugEnabled();
-      try {
-        if (isDebugEnabled) {
-          logger.debug("notifying sql disconnect listener");
-        }
-        this.sqlfDisconnectListener.onDisconnect(this);
-      } catch (VirtualMachineError e) {
-        SystemFailure.initiateFailure(e);
-        throw e;
-      } catch (Throwable e) {
-        SystemFailure.checkFailure();
-        // TODO: should these be logged or ignored?  We need to see them
-        logger.info("", e);
-      }
-      if (isDebugEnabled) {
-        logger.debug("finished notifying sql disconnect listener");
-      }
-    }
-  }
-  
-  
 
   /**
    * Makes note of a <code>DisconnectListener</code> whose
@@ -2485,12 +2448,9 @@ public class InternalDistributedSystem
           }
 
           if (isDebugEnabled) {
-            logger.debug("tryReconnect: forcedDisconnect={} sqlf listener={}", forcedDisconnect, this.sqlfDisconnectListener);
+            logger.debug("tryReconnect: forcedDisconnect={}", forcedDisconnect);
           }
           if (forcedDisconnect) {
-            // allow the fabric-service to stop before dismantling everything
-            notifySqlfForcedDisconnectListener();
-
             if (config.getDisableAutoReconnect()) {
               if (isDebugEnabled) {
                 logger.debug("tryReconnect: auto reconnect after forced disconnect is disabled");
@@ -2717,7 +2677,6 @@ public class InternalDistributedSystem
 
         DM newDM = this.reconnectDS.getDistributionManager();
         if (newDM instanceof DistributionManager) {
-          // sqlfire will have already replayed DDL and recovered.
           // Admin systems don't carry a cache, but for others we can now create
           // a cache
           if (((DistributionManager)newDM).getDMType() != DistributionManager.ADMIN_ONLY_DM_TYPE) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java
index 49e11df..21171b2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/ReplyProcessor21.java
@@ -609,8 +609,7 @@ public class ReplyProcessor21
   }
 
   // start waiting for replies without explicitly waiting for all of them using
-  // waitForReplies* methods; useful for streaming of results in function
-  // execution and SQLFabric
+  // waitForReplies* methods; useful for streaming of results in function execution
   public final void startWait() {
     if (!this.waiting && stillWaiting()) {
       preWait();
@@ -618,8 +617,7 @@ public class ReplyProcessor21
   }
 
   // end waiting for replies without explicitly invoking waitForReplies*
-  // methods; useful for streaming of results in function execution and
-  // SQLFabric
+  // methods; useful for streaming of results in function execution
   public final void endWait(boolean doCleanup) {
     try {
       postWait();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/DSCODE.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/DSCODE.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/DSCODE.java
index cef660a..583b2ab 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/DSCODE.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/DSCODE.java
@@ -322,25 +322,13 @@ public interface DSCODE {
    */
   public static final byte HUGE_STRING = 89;
 
-  /**
-   * A header byte meaning that the next element in the stream is an
-   * SQLFabric DataValueDescriptor array.
-   * 
-   * @since GemFire 6.0
-   */
-  public static final byte SQLF_DVD_ARR = 90;
+  // 90 unused
 
   /** A header byte meaning that the next element in the stream is a
    * <code>byte[][]</code>. */
   public static final byte ARRAY_OF_BYTE_ARRAYS = 91;
 
-  /**
-   * A header byte meaning that the next element in the stream is an
-   * object of SQLFabric XML type.
-   * 
-   * @since GemFire 6.5
-   */
-  public static final byte SQLF_XML = 92;
+  // 92 unused
 
   /**
    * A header byte meaning that the next element in the stream is a

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java
index 5f0002a..ab76d5c 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/DSFIDFactory.java
@@ -1064,16 +1064,6 @@ public final class DSFIDFactory implements DataSerializableFixedID {
         return readUndefined(in);
       case RESULTS_BAG:
         return readResultsBag(in);
-      case SQLF_TYPE:
-        return readSqlfMessage(in);
-      case SQLF_DVD_OBJECT:
-        return readDVD(in);
-      case SQLF_GLOBAL_ROWLOC:
-        return readGlobalRowLocation(in);
-      case SQLF_GEMFIRE_KEY:
-        return readGemFireKey(in);
-      case SQLF_FORMATIBLEBITSET:
-        return readSqlFormatibleBitSet(in);
       case TOKEN_INVALID:
         return Token.INVALID;
       case TOKEN_LOCAL_INVALID:
@@ -1175,183 +1165,6 @@ public final class DSFIDFactory implements DataSerializableFixedID {
     serializable.fromData(in);
     return serializable;
   }
-  /**
-   * Map for SQLFabric specific classIds to the {@link Class} of an
-   * implementation. We maintain this separate map for SQLFabric to allow
-   * separation of GemFire and SQLFabric trees. This is particularly required
-   * when implementing a new <code>DistributionMessage</code>. This requires the
-   * classes to have a zero argument constructor.
-   */
-  @SuppressWarnings("unchecked")
-  private static Class<? extends DataSerializableFixedID>[] sqlfDSFIDClassMap =
-    new Class[Byte.MAX_VALUE + 1 - Byte.MIN_VALUE];
-  
-  /**
-   * Map for SQLFabric specific classIds to the {@link DataSerializableFixedID} 
-   * singleton instance. We maintain this separate map for SQLFabric to allow
-   * separation of GemFire and SQLFabric trees. This approach is needed to 
-   * allow transparent serialization of singleton objects
-   */
-  private static DataSerializableFixedID[] sqlfDSFIDFixedInstanceMap =
-    new DataSerializableFixedID[Byte.MAX_VALUE + 1 - Byte.MIN_VALUE];
-
-  /**
-   * Exception to indicate SQLFabric specific serialization exceptions
-   */
-  public static class SqlfSerializationException extends
-      NotSerializableException {
-
-    private static final long serialVersionUID = 5076687296705595933L;
-
-    /**
-     * Constructs a SqlfSerializationException object with message string.
-     * 
-     * @param msg
-     *          exception message
-     */
-    public SqlfSerializationException(String msg) {
-      super(msg);
-    }
-  }
-
-  private static DataSerializableFixedID readSqlfMessage(DataInput in)
-      throws IOException, ClassNotFoundException {
-    // Use the first byte as the typeId of SQLFabric messages
-    final byte sqlfId = in.readByte();
-    final int sqlfIdIndex = sqlfId & 0xFF;
-    final Class<? extends DataSerializableFixedID> sqlfClass =
-      sqlfDSFIDClassMap[sqlfIdIndex];
-    if (sqlfClass != null) {
-      try {
-        final DataSerializableFixedID sqlfObj = sqlfClass.newInstance();
-        InternalDataSerializer.invokeFromData(sqlfObj, in);
-        return sqlfObj;
-      } catch (InstantiationException ex) {
-        throw new SqlfSerializationException(LocalizedStrings.
-            DSFIDFactory_COULD_NOT_INSTANTIATE_SQLFABRIC_MESSAGE_CLASSID_0_1
-              .toLocalizedString(new Object[] { sqlfId, ex }));
-      } catch (IllegalAccessException ex) {
-        throw new SqlfSerializationException(LocalizedStrings.
-            DSFIDFactory_ILLEGAL_ACCESS_FOR_SQLFABRIC_MESSAGE_CLASSID_0_1
-              .toLocalizedString(new Object[] { sqlfId, ex }));
-      }
-    }//check for fixed instance
-    DataSerializableFixedID fixedInstance = sqlfDSFIDFixedInstanceMap[sqlfIdIndex];
-    if (fixedInstance != null) {
-      InternalDataSerializer.invokeFromData(fixedInstance, in);
-      return fixedInstance;
-    }
-    // if possible set the processor ID before throwing exception so
-    // that failure exception is received by the sender
-    if (sqlfIdIndex < 60) {
-      try {
-        // both SqlfMessage and SqlfReplyMessage write a byte for status first
-        // followed by the processor ID, if any
-        final byte status = in.readByte();
-        int processorId = 0;
-        if ((status & ReplyMessage.PROCESSOR_ID_FLAG) != 0) {
-          processorId = in.readInt();
-        }
-        ReplyProcessor21.setMessageRPId(processorId);
-      } catch (IOException ex) {
-        // give up
-      }
-    }
-    throw new SqlfSerializationException(
-        LocalizedStrings.DSFIDFactory_UNKNOWN_CLASSID_0_FOR_SQLFABRIC_MESSAGE
-            .toLocalizedString(sqlfId));
-  }
-
-  public static synchronized void registerSQLFabricClass(byte classId,
-      Class<? extends DataSerializableFixedID> c) {
-    final int sqlfIdIndex = classId & 0xFF;
-    Class<?> oldClass = sqlfDSFIDClassMap[sqlfIdIndex];
-    if (oldClass != null) {
-      throw new AssertionError("DSFIDFactory#registerSQLFabricClass: cannot "
-          + "re-register classId " + classId + " for class " + c
-          + "; existing class: " + oldClass);
-    }
-    sqlfDSFIDClassMap[sqlfIdIndex] = c;
-  }
-  
-  public static synchronized void registerSQLFabricFixedInstance(byte classId,
-      DataSerializableFixedID fixedInstance)
-  {
-    final int sqlfIdIndex = classId & 0xFF;
-    DataSerializableFixedID oldInstance = sqlfDSFIDFixedInstanceMap[sqlfIdIndex];
-    if (oldInstance != null) {
-      throw new AssertionError("DSFIDFactory#registerSQLFabricClass: cannot "
-          + "re-register classId " + classId + " for instance " + fixedInstance
-          + "; existing instance: " + oldInstance);
-    }
-    sqlfDSFIDFixedInstanceMap[sqlfIdIndex] = fixedInstance;
-  }
-
-  public static synchronized void unregisterSQLFabricClass(byte classId,
-      Class<? extends DataSerializableFixedID> c) {
-    final int sqlfIdIndex = classId & 0xFF;
-    sqlfDSFIDClassMap[sqlfIdIndex] = null;
-  }
-  
-  public static synchronized void unregisterSQLFabricFixedInstance(
-      byte classId, Object dataSerializableFixedID)
-  {
-    final int sqlfIdIndex = classId & 0xFF;
-    sqlfDSFIDFixedInstanceMap[sqlfIdIndex] = null;
-  }
-
-  public static synchronized void clearSQLFabricClasses() {
-    for (int index = 0; index < sqlfDSFIDClassMap.length; ++index) {
-      sqlfDSFIDClassMap[index] = null;
-    }
-    for (int index = 0; index < sqlfDSFIDFixedInstanceMap.length; ++index) {
-      sqlfDSFIDFixedInstanceMap[index] = null;
-    }
-  }  
-
-  public interface DeserializeDVD {
-
-    public DataSerializableFixedID getDSFID(DataInput in) throws IOException,
-        ClassNotFoundException;
-
-    public DataSerializableFixedID getGlobalRowLocation(DataInput in)
-        throws IOException, ClassNotFoundException;
-
-    public DataSerializableFixedID getGemFireKey(DataInput in)
-        throws IOException, ClassNotFoundException;
-
-    public DataSerializableFixedID getSqlPSQArgs(DataInput in)
-        throws IOException, ClassNotFoundException;
-
-    public DataSerializableFixedID getSqlFormatibleBitSet(DataInput in)
-        throws IOException, ClassNotFoundException;
-  }
-
-  private static DeserializeDVD dvdDeserializer;
-
-  private static DataSerializableFixedID readDVD(DataInput in)
-      throws IOException, ClassNotFoundException {
-    return dvdDeserializer.getDSFID(in);
-  }
-
-  private static DataSerializableFixedID readGlobalRowLocation(DataInput in)
-      throws IOException, ClassNotFoundException {
-    return dvdDeserializer.getGlobalRowLocation(in);
-  }
-
-  private static DataSerializableFixedID readGemFireKey(DataInput in)
-      throws IOException, ClassNotFoundException {
-    return dvdDeserializer.getGemFireKey(in);
-  }
-
-  private static DataSerializableFixedID readSqlFormatibleBitSet(DataInput in)
-      throws IOException, ClassNotFoundException {
-    return dvdDeserializer.getSqlFormatibleBitSet(in);
-  }
-
-  public static void registerDVDDeserializer(DeserializeDVD d) {
-    dvdDeserializer = d;
-  }
 
   private static DataSerializableFixedID readSnappyCompressedCachedDeserializable(DataInput in) 
   throws IOException, ClassNotFoundException {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java
index 0788503..d3e4846 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/DataSerializableFixedID.java
@@ -136,8 +136,6 @@ public interface DataSerializableFixedID extends SerializationVersions {
   public static final byte ADD_CACHESERVER_PROFILE_UPDATE = -118;  
   public static final byte SERVER_INTEREST_REGISTRATION_MESSAGE = -117;
   public static final byte FILTER_PROFILE_UPDATE = -116;
-  // [sumedh] below two IDs are no longer used in new TX model and will be
-  // removed at some point after SQLF upmerge
   public static final byte JTA_AFTER_COMPLETION_MESSAGE = -115;
   public static final byte JTA_BEFORE_COMPLETION_MESSAGE = -114;
   public static final byte INVALIDATE_PARTITIONED_REGION_MESSAGE = -113;
@@ -169,8 +167,6 @@ public interface DataSerializableFixedID extends SerializationVersions {
 
   public static final byte CREATE_REGION_MESSAGE = -89;
   public static final byte DESTROY_PARTITIONED_REGION_MESSAGE = -88;
-  // [sumedh] below two IDs are no longer used in new TX model and will be
-  // removed at some point after SQLF upmerge
   public static final byte COMMIT_PROCESS_QUERY_MESSAGE = -87;
   public static final byte COMMIT_PROCESS_QUERY_REPLY_MESSAGE = -86;
   public static final byte DESTROY_REGION_WITH_CONTEXT_MESSAGE = -85;
@@ -182,8 +178,6 @@ public interface DataSerializableFixedID extends SerializationVersions {
   public static final byte STATE_STABILIZATION_MESSAGE = -79;
   public static final byte STATE_STABILIZED_MESSAGE = -78;
   public static final byte CLIENT_MARKER_MESSAGE_IMPL = -77;
-  // [sumedh] below three IDs are no longer used in new TX model and will be
-  // removed at some point after SQLF upmerge
   public static final byte TX_LOCK_UPDATE_PARTICIPANTS_MESSAGE = -76;
   public static final byte TX_ORIGINATOR_RECOVERY_MESSAGE = -75;
   public static final byte TX_ORIGINATOR_RECOVERY_REPLY_MESSAGE = -74;
@@ -193,8 +187,6 @@ public interface DataSerializableFixedID extends SerializationVersions {
   public static final byte NON_GRANTOR_DESTROYED_REPLY_MESSAGE = -70;
   public static final byte TOMBSTONE_MESSAGE = -69;
   public static final byte IDS_REGISTRATION_MESSAGE = -68;
-  // [sumedh] below ID is no longer used in new TX model and will be
-  // removed at some point after SQLF upmerge
   public static final byte TX_LOCK_UPDATE_PARTICIPANTS_REPLY_MESSAGE = -67;
   public static final byte STREAMING_REPLY_MESSAGE = -66;
   public static final byte PREFER_BYTES_CACHED_DESERIALIZABLE = -65;
@@ -223,8 +215,6 @@ public interface DataSerializableFixedID extends SerializationVersions {
   public static final byte GET_ALL_SERVERS_REQUEST = -43;
   public static final byte GET_ALL_SERVRES_RESPONSE = -42;
 
-  // [sumedh] below two IDs are no longer used in new TX model and will be
-  // removed at some point after SQLF upmerge
   public static final byte FIND_REMOTE_TX_REPLY = -41;
   public static final byte FIND_REMOTE_TX_MESSAGE = -40;
 
@@ -257,40 +247,7 @@ public interface DataSerializableFixedID extends SerializationVersions {
   
   public static final byte CLIENT_INTEREST_MESSAGE = -21;
 
-  /**
-   * A header byte meaning that the next element in the stream is a
-   * type meant for SQL Fabric.
-   */
-  public static final byte SQLF_TYPE = -20;
-
-  /**
-   * A header byte meaning that the next element in the stream is a
-   * DVD object used for SQL Fabric.
-   */
-  public static final byte SQLF_DVD_OBJECT = -19;
-  
-  /**
-   * A header byte meaning that the next element in the stream is a
-   * GlobalRowLocation object used for SQL Fabric.
-   */
-  public static final byte SQLF_GLOBAL_ROWLOC = -18;
-  
-  /**
-   * A header byte meaning that the next element in the stream is a
-   * GemFireKey object used for SQL Fabric.
-   */
-  public static final byte SQLF_GEMFIRE_KEY = -17;
-  
-  /**
-   * A header byte meaning that the next element in the stream is a
-   * FormatibleBitSet object in SQLFabric.
-   */
-  public static final byte SQLF_FORMATIBLEBITSET = -16;
-
-  // IDs -15 .. -10 are not used in trunk yet but only in SQLFire, so marking
-  // as used so that GemFire does not use them until the SQLF upmerge else
-  // there will be big problems in backward compatibility after upmerge which
-  // is required for both >= SQLF 1.1 and >= GFE 7.1
+  // IDs -20 .. -16 are not used
 
   /**
    * A header byte meaning that the next element in the stream is a
@@ -541,8 +498,6 @@ public interface DataSerializableFixedID extends SerializationVersions {
   // TXId
   public static final byte TRANSACTION_ID = 109;
 
-  // [sumedh] below ID is no longer used in new TX model and will be
-  // removed at some point after SQLF upmerge
   public static final byte TX_COMMIT_MESSAGE = 110;
 
   public static final byte HA_PROFILE = 111;
@@ -565,8 +520,6 @@ public interface DataSerializableFixedID extends SerializationVersions {
 
   public static final byte PR_GET_MESSAGE = 120;
 
-  // [sumedh] below two IDs are no longer used in new TX model and will be
-  // removed at some point after SQLF upmerge
   // TXLockIdImpl
   public static final byte TRANSACTION_LOCK_ID = 121;
   // TXCommitMessage.CommitProcessForLockIdMessage
@@ -876,9 +829,6 @@ public interface DataSerializableFixedID extends SerializationVersions {
    * class. e.g. if msg format changed in version 80, create toDataPre_GFE_8_0_0_0, add
    * Version.GFE_80 to the getSerializationVersions array and copy previous toData contents 
    * to this newly created toDataPre_GFE_X_X_X_X() method.
-   * <p>
-   * For GemFireXD use "GFXD" (or whatever we decide on as a product identifier
-   * in Version) instead of "GFE" in method names.
    * @throws IOException
    *           A problem occurs while writing to <code>out</code>
    */
@@ -894,10 +844,6 @@ public interface DataSerializableFixedID extends SerializationVersions {
    * class. e.g. if msg format changed in version 80, create fromDataPre_GFE_8_0_0_0, add
    * Version.GFE_80 to the getSerializationVersions array  and copy previous fromData 
    * contents to this newly created fromDataPre_GFE_X_X_X_X() method.
-   * <p>
-   * For GemFireXD use "GFXD" (or whatever we decide on as a product identifier
-   * in Version) instead of "GFE" in method names.
-   * 
    * @throws IOException
    *           A problem occurs while reading from <code>in</code>
    * @throws ClassNotFoundException

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/GemFireUtilLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/GemFireUtilLauncher.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/GemFireUtilLauncher.java
deleted file mode 100644
index f7a3628..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/GemFireUtilLauncher.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package com.gemstone.gemfire.internal;
-
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.admin.jmx.internal.AgentLauncher;
-import com.gemstone.gemfire.internal.SystemAdmin;
-import com.gemstone.gemfire.internal.cache.CacheServerLauncher;
-
-/**
- * Maps the GemFire utilities to the launcher that starts them and then invokes
- * that class's main method. Currently this class is only a base class for the 
- * SqlFabric implementation, but eventually the gemfire scripts will be 
- * consolidated to use this class.
- * Current GemFire utilities (as of 6.0):
- * <ul>
- * <li> agent 
- * <li> gemfire
- * <li> cacheserver 
- * </ul>
- * Usage:
- * notYetWritenScript <utility> <utility arguments>
- *
- * @since GemFire 6.0
- */
-public class GemFireUtilLauncher {
-
-  /**
-   * Returns a mapping of utility names to the class used to spawn them.
-   * This method is overridedn by SqlFabricUtilLauncher to handle that product's
-   * own utility tools.
-   **/
-  protected Map<String, Class<?>> getTypes() {
-    Map<String, Class<?>> m = new HashMap<String, Class<?>>();
-    m.put("agent", AgentLauncher.class);
-    m.put("gemfire", SystemAdmin.class);
-    m.put("cacheserver", CacheServerLauncher.class);
-    return m;
-  }
-
-  /** 
-   * A simple constructor was needed so that {@link #usage(String)} 
-   * and {@link #getTypes()} could be non-static methods.
-   **/
-  protected GemFireUtilLauncher() {}
-
-  /** 
-   * This method should be overridden if the name of the script is different.
-   * @return the name of the script used to launch this utility. 
-   **/
-  protected String scriptName() {
-    return "gemfire"; 
-  }
-
-  /** 
-   * Print help information for this utility.
-   * This method is intentionally non-static so that getTypes() can dynamically
-   * display the list of supported utilites supported by child classes.
-   * @param context print this message before displaying the regular help text
-   **/
-  private void usage(String context) {
-    System.out.println(context);
-    StringBuffer sb = new StringBuffer();
-    sb.append("help|");
-    for(String key : getTypes().keySet()) {
-      sb.append(key).append("|");
-    }
-    sb.deleteCharAt(sb.length()-1); // remove the extra "|"
-    String msg = LocalizedStrings.GemFireUtilLauncher_ARGUMENTS
-                   .toLocalizedString(new Object[] {scriptName(), sb});
-    System.out.println(msg);
-    System.exit(1);
-  }
-
-  /**
-   * Spawn the utilty passed in as args[0] or display help information
-   * @param args a utilty and the arguments to pass to it.
-   */
-  public static void main(String[] args) {
-    GemFireUtilLauncher launcher = new GemFireUtilLauncher();
-        launcher.validateArgs(args);
-    launcher.invoke(args);
-  }
-  
-  /**
-   * Calls the <code>public static void main(String[] args)</code> method
-   * of the class associated with the utility name.  
-   * @param args the first argument is the utility name, the remainder 
-   *             comprises the arguments to be passed
-   */
-  protected void invoke(String[] args) {
-    Class<?> clazz = getTypes().get(args[0]);
-    if(clazz == null) {
-      usage(LocalizedStrings.GemFireUtilLauncher_INVALID_UTILITY_0
-            .toLocalizedString(args[0]));
-    }
-    int len = args.length-1;
-    String[] argv = new String[len];
-    System.arraycopy(args, 1, argv, 0, len);
-    
-    Exception ex = null;
-    try {
-      Method m = clazz.getDeclaredMethod("main", new Class[] {argv.getClass()});
-      m.invoke(null, (Object)argv);
-    } catch (SecurityException se) {
-      ex = se;
-    } catch (NoSuchMethodException nsme) {
-      ex = nsme;    
-    } catch (IllegalArgumentException iae) {
-      ex = iae;
-    } catch (IllegalAccessException iae) {
-      ex = iae;
-    } catch (InvocationTargetException ite) {
-      ex = ite;
-    } finally {
-      if (ex != null) {
-        String msg = LocalizedStrings.GemFireUtilLauncher_PROBLEM_STARTING_0
-                                     .toLocalizedString(args[0]); 
-        throw new RuntimeException(msg, ex);
-      }
-    }
-  }
- 
-  /**
-   * Look for variations on help and validate the arguments make sense.
-   * A usage mesage is displayed if necesary.
-   * The following forms of help are accepted:
-   * <code>--help, -help, /help, --h, -h, /h</code>
-   **/ 
-  protected void validateArgs(String[] args) {
-    if (args.length == 0) {
-      usage(LocalizedStrings.GemFireUtilLauncher_MISSING_COMMAND
-                            .toLocalizedString());
-    }
-    //Match all major variations on --help
-    Pattern help = Pattern.compile("(?:--|-|/){0,1}h(?:elp)*", 
-        Pattern.UNICODE_CASE | Pattern.CASE_INSENSITIVE);
-    Matcher matcher = help.matcher(args[0]); 
-
-    if( matcher.matches() ) {
-      usage(LocalizedStrings.GemFireUtilLauncher_HELP.toLocalizedString());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java
index 0df656a..33cd410 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/InternalDataSerializer.java
@@ -74,11 +74,6 @@ public abstract class InternalDataSerializer extends DataSerializer implements D
    */
   private static final ConcurrentHashMap<String, DataSerializer> classesToSerializers = new ConcurrentHashMap<String, DataSerializer>();
   
-  // used by sqlFire
-  public static ConcurrentHashMap<String, DataSerializer> getClassesToSerializers() {
-    return classesToSerializers;
-  }
-
   private static final String serializationVersionTxt = System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "serializationVersion");
   /**
    * Any time new serialization format is added then a new enum needs to be added here.
@@ -2791,8 +2786,6 @@ public abstract class InternalDataSerializer extends DataSerializer implements D
       return DSFIDFactory.create(in.readInt(), in);
     case DS_NO_FIXED_ID:
       return readDataSerializableFixedID(in);
-    case SQLF_DVD_ARR:
-      return dvddeserializer.fromData(in);
     case NULL:
       return null;
     case NULL_STRING:

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/SystemAdmin.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/SystemAdmin.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/SystemAdmin.java
index 7cb7df1..5e561ed 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/SystemAdmin.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/SystemAdmin.java
@@ -1256,8 +1256,6 @@ public class SystemAdmin {
   }
 
   public SystemAdmin() {
-    // no instances allowed
-    // [sumedh] now is overridden by SQLF
     // register DSFID types first; invoked explicitly so that all message type
     // initializations do not happen in first deserialization on a possibly
     // "precious" thread

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/Version.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/Version.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/Version.java
index 513b5bd..1b8543e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/Version.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/Version.java
@@ -56,7 +56,7 @@ public final class Version implements Comparable<Version> {
 
   /**
    * Set to non-null if the underlying GemFire version is different from product
-   * version (e.g. for SQLFire)
+   * version
    */
   private Version gemfireVersion;
 
@@ -142,19 +142,6 @@ public final class Version implements Comparable<Version> {
   public static final Version GFE_701 = new Version("GFE", "7.0.1", (byte)7,
       (byte)0, (byte)1, (byte)0, GFE_701_ORDINAL);
 
-  /**
-   * SQLFire 1.1 has a separate version since it has changed the RowFormatter
-   * formatting for ALTER TABLE add/drop column support. However, its underlying
-   * GemFire version will remain at GFE_7x.
-   * 
-   * This version is an intermediate one created to test rolling upgrades. It is
-   * compatible with <code>SQLF_11</code> in all respects except for artifical
-   * changes in a couple of P2P messages and marking as compatible with GFE_701.
-   * 
-   * This is the GemFire conterpart of SQLF_1099 for testing rolling upgrades
-   * and it uses the same ordinal as GFE_701 to maintain compatibility with the
-   * ordinals being used on SQLFire branch.
-   */
   private static final byte GFE_7099_ORDINAL = 21;
 
   public static final Version GFE_7099 = new Version("GFE", "7.0.99", (byte)7,
@@ -199,7 +186,7 @@ public final class Version implements Comparable<Version> {
       (byte)0, (byte)0, (byte)0, GFE_90_ORDINAL);
 
   /**
-   * This constant must be set to the most current version of GFE/SQLF.
+   * This constant must be set to the most current version of the product.
    * !!! NOTE: update HIGHEST_VERSION when changing CURRENT !!!
    */
   public static final Version CURRENT = GFE_90;
@@ -261,8 +248,8 @@ public final class Version implements Comparable<Version> {
     if (ordinal == TOKEN_ORDINAL) {
       return TOKEN;
     }
-    // for GFE clients also check that there must be a commands object mapping
-    // for processing (SQLF product versions will not work)
+    // for clients also check that there must be a commands object mapping
+    // for processing
     if ((VALUES.length < ordinal + 1) || VALUES[ordinal] == null
         || (forGFEClients && CommandInitializer.getCommands(VALUES[ordinal]) == null)) {
       throw new UnsupportedVersionException(

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/VersionedDataStream.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/VersionedDataStream.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/VersionedDataStream.java
index 7c5ca11..3e8bd98 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/VersionedDataStream.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/VersionedDataStream.java
@@ -25,10 +25,7 @@ import com.gemstone.gemfire.DataSerializable;
 /**
  * An extension to {@link DataOutput}, {@link DataInput} used internally in
  * product to indicate that the input/output stream is attached to a GemFire
- * peer having a different version. See the spec on rolling upgrades for more
- * details: <a
- * href="https://wiki.gemstone.com/display/SQLF/Rolling+upgrades">Rolling
- * Upgrades</a>.
+ * peer having a different version.
  * 
  * Internal product classes that implement {@link DataSerializableFixedID} and
  * {@link DataSerializable} and change serialization format must check this on

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractDiskRegionEntry.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractDiskRegionEntry.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractDiskRegionEntry.java
index b65b7ad..41cd110 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractDiskRegionEntry.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractDiskRegionEntry.java
@@ -67,9 +67,4 @@ public abstract class AbstractDiskRegionEntry
       GatewaySenderEventImpl.release(this._getValue()); // OFFHEAP _getValue ok
     }
   }
-  @Override
-  public void afterValueOverflow(RegionEntryContext context) {
-    //NO OP
-    //Overridden in sqlf RegionEntry
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java
index 35f16bc..6ee4c17 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionEntry.java
@@ -150,7 +150,7 @@ public abstract class AbstractRegionEntry implements RegionEntry,
         // by the RegionMap. It is unclear why this code is needed. ARM destroy
         // does this also and we are now doing it as phase3 of the ARM destroy.
         removePhase2();
-        rgn.getRegionMap().removeEntry(event.getKey(), this, true, event, rgn, rgn.getIndexUpdater());
+        rgn.getRegionMap().removeEntry(event.getKey(), this, true, event, rgn);
       }
     }
   }
@@ -291,10 +291,6 @@ public abstract class AbstractRegionEntry implements RegionEntry,
       }
     }
 
-    final boolean isEagerDeserialize = dst.isEagerDeserialize();
-    if (isEagerDeserialize) {
-      dst.clearEagerDeserialize();
-    }
     dst.setLastModified(mgr, getLastModified()); // fix for bug 31059
     if (v == Token.INVALID) {
       dst.setInvalid();
@@ -307,17 +303,11 @@ public abstract class AbstractRegionEntry implements RegionEntry,
     }
     else if (v instanceof CachedDeserializable) {
       // don't serialize here if it is not already serialized
-//      if(v instanceof ByteSource && CachedDeserializableFactory.preferObject()) {
-//        // For SQLFire we prefer eager deserialized
-//        dst.setEagerDeserialize();         
-//      }
       CachedDeserializable cd = (CachedDeserializable) v;
       if (!cd.isSerialized()) {
         dst.value = cd.getDeserializedForReading();
       } else {
-        /*if (v instanceof ByteSource && CachedDeserializableFactory.preferObject()) {
-          dst.value = v;
-        } else */ {
+        {
           Object tmp = cd.getValue();
           if (tmp instanceof byte[]) {
             byte[] bb = (byte[]) tmp;
@@ -352,11 +342,7 @@ public abstract class AbstractRegionEntry implements RegionEntry,
           return false;
         }
       }
-    if (CachedDeserializableFactory.preferObject()) {
-      dst.value = preparedValue;
-      dst.setEagerDeserialize();
-    }
-    else {
+    {
       try {
         HeapDataOutputStream hdos = new HeapDataOutputStream(Version.CURRENT);
         BlobHelper.serializeTo(preparedValue, hdos);
@@ -412,7 +398,7 @@ public abstract class AbstractRegionEntry implements RegionEntry,
       ReferenceCountHelper.setReferenceCountOwner(null);
       return null;
     } else {
-      result = OffHeapHelper.copyAndReleaseIfNeeded(result); // sqlf does not dec ref count in this call
+      result = OffHeapHelper.copyAndReleaseIfNeeded(result);
       ReferenceCountHelper.setReferenceCountOwner(null);
       setRecentlyUsed();
       return result;
@@ -749,9 +735,7 @@ public abstract class AbstractRegionEntry implements RegionEntry,
       } 
       else {
         FilterProfile fp = region.getFilterProfile();
-        // rdubey: Old value also required for SqlfIndexManager.
-        if (fp != null && ((fp.getCqCount() > 0) || expectedOldValue != null
-            || event.getRegion().getIndexUpdater() != null)) {
+        if (fp != null && ((fp.getCqCount() > 0) || expectedOldValue != null)) {
           //curValue = getValue(region); can cause deadlock will fault in the value
           // and will confuse LRU. rdubey.
           curValue = getValueOnDiskOrBuffer(region);
@@ -1393,10 +1377,6 @@ public abstract class AbstractRegionEntry implements RegionEntry,
       }
     }
   }
-  /**
-   * soubhik: this method is overridden in sqlf flavor of entries.
-   * Instead of overriding this method; override areSetValue.
-   */
   protected final void _setValue(Object val) {
     setValueField(val);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
index a512750..bc919fc 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractRegionMap.java
@@ -22,7 +22,6 @@ import com.gemstone.gemfire.InvalidDeltaException;
 import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.cache.query.IndexMaintenanceException;
 import com.gemstone.gemfire.cache.query.QueryException;
-import com.gemstone.gemfire.cache.query.internal.IndexUpdater;
 import com.gemstone.gemfire.cache.query.internal.index.IndexManager;
 import com.gemstone.gemfire.cache.query.internal.index.IndexProtocol;
 import com.gemstone.gemfire.distributed.DistributedMember;
@@ -31,7 +30,6 @@ import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedM
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.cache.DiskInitFile.DiskRegionFlag;
 import com.gemstone.gemfire.internal.cache.FilterRoutingInfo.FilterInfo;
-import com.gemstone.gemfire.internal.cache.delta.Delta;
 import com.gemstone.gemfire.internal.cache.ha.HAContainerWrapper;
 import com.gemstone.gemfire.internal.cache.ha.HARegionQueue;
 import com.gemstone.gemfire.internal.cache.lru.LRUEntry;
@@ -69,21 +67,12 @@ import java.util.concurrent.atomic.AtomicInteger;
  *
  */
 
-//Asif: In case of sqlFabric System, we are creating a different set of RegionEntry 
-// which are derived from the concrete  GFE RegionEntry classes.
-// In future if any new concrete  RegionEntry class is defined, the new  SqlFabric
-// RegionEntry Classes need to be created. There is a junit test in sqlfabric
-// which checks for RegionEntry classes of GFE and validates the same with its 
-// own classes.
-
 public abstract class AbstractRegionMap implements RegionMap {
 
   private static final Logger logger = LogService.getLogger();
   
   /** The underlying map for this region. */
   protected CustomEntryConcurrentHashMap<Object, Object> map;
-  /** An internal Listener for index maintenance for SQLFabric. */
-  private final IndexUpdater indexUpdater;
 
   /**
    * This test hook is used to force the conditions for defect 48182.
@@ -96,16 +85,6 @@ public abstract class AbstractRegionMap implements RegionMap {
   private transient Object owner; // the region that owns this map
   
   protected AbstractRegionMap(InternalRegionArguments internalRegionArgs) {
-    if (internalRegionArgs != null) {
-      this.indexUpdater = internalRegionArgs.getIndexUpdater();
-    }
-    else {
-      this.indexUpdater = null;
-    }
-  }
-
-  public final IndexUpdater getIndexUpdater() {
-    return this.indexUpdater;
   }
 
   protected void initialize(Object owner,
@@ -299,28 +278,17 @@ public abstract class AbstractRegionMap implements RegionMap {
   }
 
   public final void removeEntry(Object key, RegionEntry re, boolean updateStat,
-      EntryEventImpl event, final LocalRegion owner,
-      final IndexUpdater indexUpdater) {
+      EntryEventImpl event, final LocalRegion owner) {
     boolean success = false;
     if (re.isTombstone()&& _getMap().get(key) == re) {
       logger.fatal(LocalizedMessage.create(LocalizedStrings.AbstractRegionMap_ATTEMPT_TO_REMOVE_TOMBSTONE), new Exception("stack trace"));
       return; // can't remove tombstones except from the tombstone sweeper
     }
-    try {
-      if (indexUpdater != null) {
-        indexUpdater.onEvent(owner, event, re);
-      }
-
-      if (_getMap().remove(key, re)) {
-        re.removePhase2();
-        success = true;
-        if (updateStat) {
-          incEntryCount(-1);
-        }
-      }
-    } finally {
-      if (indexUpdater != null) {
-        indexUpdater.postEvent(owner, event, re, success);
+    if (_getMap().remove(key, re)) {
+      re.removePhase2();
+      success = true;
+      if (updateStat) {
+        incEntryCount(-1);
       }
     }
   }
@@ -759,7 +727,6 @@ public abstract class AbstractRegionMap implements RegionMap {
                                        boolean deferLRUCallback,
                                        VersionTag entryVersion, InternalDistributedMember sender, boolean isSynchronizing)
   {
-    assert indexUpdater == null : "indexUpdater should only exist if sqlfire";
     boolean result = false;
     boolean done = false;
     boolean cleared = false;
@@ -1377,8 +1344,7 @@ public abstract class AbstractRegionMap implements RegionMap {
                       if (!inTokenMode) {
                         if ( re.getVersionStamp() == null) {
                           re.removePhase2();
-                          removeEntry(event.getKey(), re, true, event, owner,
-                              indexUpdater);
+                          removeEntry(event.getKey(), re, true, event, owner);
                           removed = true;
                         }
                       }
@@ -1399,8 +1365,7 @@ public abstract class AbstractRegionMap implements RegionMap {
                         owner.recordEvent(event);
                         if (re.getVersionStamp() == null) {
                           re.removePhase2();
-                          removeEntry(event.getKey(), re, true, event, owner,
-                              indexUpdater);
+                          removeEntry(event.getKey(), re, true, event, owner);
                           lruEntryDestroy(re);
                         } else {
                           if (re.isTombstone()) {
@@ -1430,8 +1395,7 @@ public abstract class AbstractRegionMap implements RegionMap {
                   finally {
                     if (re.isRemoved() && !re.isTombstone()) {
                       if (!removed) {
-                        removeEntry(event.getKey(), re, true, event, owner,
-                            indexUpdater);
+                        removeEntry(event.getKey(), re, true, event, owner);
                       }
                     }
                   }
@@ -1543,7 +1507,6 @@ public abstract class AbstractRegionMap implements RegionMap {
         try {
           synchronized (re) {
             if (!re.isRemoved() || re.isTombstone()) {
-              EntryEventImpl sqlfEvent = null;
               Object oldValue = re.getValueInVM(owner);
               final int oldSize = owner.calculateRegionEntryValueSize(re);
               // Create an entry event only if the calling context is
@@ -1554,15 +1517,10 @@ public abstract class AbstractRegionMap implements RegionMap {
                   key, null, txId, txEvent, eventId, aCallbackArgument, filterRoutingInfo, bridgeContext, txEntryState, versionTag, tailKey);
               try {
               
-              if (/* owner.isUsedForPartitionedRegionBucket() && */ 
-                  indexUpdater != null) {
-                 sqlfEvent = cbEvent;
-              } else {
                 if (owner.isUsedForPartitionedRegionBucket()) {
                   txHandleWANEvent(owner, cbEvent, txEntryState);
                 }
                 cbEvent.setRegionEntry(re);
-              }
               cbEvent.setOldValue(oldValue);
               if (isDebugEnabled) {
                 logger.debug("txApplyDestroy cbEvent={}", cbEvent);
@@ -1583,11 +1541,7 @@ public abstract class AbstractRegionMap implements RegionMap {
                 }
                 else {
                   if (!re.isTombstone()) {
-                    if (sqlfEvent != null) {
-                      re.removePhase1(owner, false); // fix for bug 43063
-                      re.removePhase2();
-                      removeEntry(key, re, true, sqlfEvent, owner, indexUpdater);
-                    } else {
+                    {
                       if (shouldPerformConcurrencyChecks(owner, cbEvent) && cbEvent.getVersionTag() != null) {
                         re.makeTombstone(owner, cbEvent.getVersionTag());
                       } else {
@@ -2616,80 +2570,6 @@ public abstract class AbstractRegionMap implements RegionMap {
     return retVal;
   }
 
-  protected static final MapCallbackAdapter<Object, Object, Object, Object>
-      listOfDeltasCreator = new MapCallbackAdapter<Object, Object,
-          Object, Object>() {
-    @Override
-    public Object newValue(Object key, Object context, Object createParams,
-        final MapResult result) {
-      return new ListOfDeltas(4);
-    }
-  };
-  
-  /**
-   * Neeraj: The below if block is to handle the special
-   * scenario witnessed in Sqlfabric for now. (Though its
-   * a general scenario). The scenario is that the updates start coming 
-   * before the base value reaches through GII. In that scenario the updates
-   * essentially the deltas are added to a list and kept as oldValue in the
-   * map and this method returns. When through GII the actual base value arrives
-   * these updates or deltas are applied on it and the new value thus got is put
-   * in the map.
-   * @param event 
-   * @param ifOld 
-   * @return true if delta was enqued
-   */
-  private boolean enqueDelta(EntryEventImpl event, boolean ifOld) {
-    final IndexUpdater indexManager = getIndexUpdater();
-    LocalRegion owner = _getOwner();
-    if (indexManager != null && !owner.isInitialized() && event.hasDelta()) {
-      boolean isOldValueDelta = true;
-      try {
-        if (ifOld) {
-          final Delta delta = event.getDeltaNewValue();
-		  RegionEntry re = getOrCreateRegionEntry(owner, event, null,
-          	  listOfDeltasCreator, false, false);
-          assert re != null;
-          synchronized (re) {
-            @Retained @Released Object oVal = re.getValueOffHeapOrDiskWithoutFaultIn(owner);
-            if (oVal != null) {
-              try {
-              if (oVal instanceof ListOfDeltas) {
-                if (logger.isDebugEnabled()) {
-                  logger.debug("basicPut: adding delta to list of deltas: {}", delta);
-                }
-                ((ListOfDeltas)oVal).merge(delta);
-                @Retained Object newVal = ((AbstractRegionEntry)re).prepareValueForCache(owner, oVal, true);              
-                re.setValue(owner, newVal); // TODO:KIRK:48068 prevent orphan
-              }
-              else {
-                isOldValueDelta = false;
-              }
-              }finally {
-                OffHeapHelper.release(oVal);
-              }
-            }
-            else {
-              if (logger.isDebugEnabled()) {
-                logger.debug("basicPut: new list of deltas with delta: {}", delta);
-              }
-              @Retained Object newVal = new ListOfDeltas(delta);
-              // TODO no need to call AbstractRegionMap.prepareValueForCache here?
-              newVal = ((AbstractRegionEntry)re).prepareValueForCache(owner, newVal, true);
-              re.setValue(owner, newVal); // TODO:KIRK:48068 prevent orphan
-            }
-          }
-        }
-      } catch (RegionClearedException ex) {
-        // Neeraj: We can just ignore this exception because we are returning after this block
-      }
-      if (isOldValueDelta) {
-        return true;
-      }
-    }
-    return false;
-  }
-
   /*
    * returns null if the operation fails
    */
@@ -2747,31 +2627,11 @@ public abstract class AbstractRegionMap implements RegionMap {
     // reference of the diskSegmentRegion as a ThreadLocal so that if the diskRegionSegment
     // is later changed by another thread, we can do the necessary.
     boolean uninitialized = !owner.isInitialized();
-    // SqlFabric Changes - BEGIN
-    if (enqueDelta(event, ifOld)) {
-      return null;
-    }
-
-    final IndexUpdater indexManager = getIndexUpdater();
-
-    boolean sqlfIndexLocked = false;
-    // SqlFabric Changes - END
-
     boolean retrieveOldValueForDelta = event.getDeltaBytes() != null
         && event.getRawNewValue() == null;
     lockForCacheModification(owner, event);
     IndexManager oqlIndexManager = null;
     try {
-      // take read lock for SQLF index initializations if required; the index
-      // GII lock is for any updates that may come in while index is being
-      // loaded during replay see bug #41377; this will go away once we allow
-      // for indexes to be loaded completely in parallel (#40899); need to
-      // take this lock before the RegionEntry lock else a deadlock can happen
-      // between this thread and index loading thread that will first take the
-      // corresponding write lock on the IndexUpdater
-      if (indexManager != null) {
-        sqlfIndexLocked = indexManager.lockForIndexGII();
-      }
       // Fix for Bug #44431. We do NOT want to update the region and wait
       // later for index INIT as region.clear() can cause inconsistency if
       // happened in parallel as it also does index INIT.
@@ -2883,8 +2743,7 @@ public abstract class AbstractRegionMap implements RegionMap {
                 } finally {
                   OffHeapHelper.release(oldValueForDelta);
                   if (re != null && !onlyExisting && !isOpComplete(re, event)) {
-                    owner.cleanUpOnIncompleteOp(event, re, eventRecorded,
-                        false/* updateStats */, replaceOnClient);
+                    owner.cleanUpOnIncompleteOp(event, re);
                   }
                   else if (re != null && owner.isUsedForPartitionedRegionBucket()) {
                   BucketRegion br = (BucketRegion)owner;
@@ -2902,9 +2761,6 @@ public abstract class AbstractRegionMap implements RegionMap {
       throw dae;
     } finally {
         releaseCacheModificationLock(owner, event);
-        if (sqlfIndexLocked) {
-          indexManager.unlockForIndexGII();
-        }
         if (oqlIndexManager != null) {
           oqlIndexManager.countDownIndexUpdaters();
         }
@@ -2914,22 +2770,6 @@ public abstract class AbstractRegionMap implements RegionMap {
             final boolean invokeListeners = event.basicGetNewValue() != Token.TOMBSTONE;
             owner.basicPutPart3(event, result, !uninitialized,
                 lastModifiedTime, invokeListeners, ifNew, ifOld, expectedOldValue, requireOldValue);
-          } catch (EntryExistsException eee) {
-            // SQLFabric changes BEGIN
-            // ignore EntryExistsException in distribution from a non-empty
-            // region since actual check will be done in this put itself
-            // and it can happen in distribution if put comes in from
-            // GII as well as distribution channel
-            if (indexManager != null) {
-              if (logger.isTraceEnabled()) {
-                logger.trace("basicPut: ignoring EntryExistsException in distribution {}", eee);
-              }
-            }
-            else {
-              // can this happen for non-SQLFabric case?
-              throw eee;
-            }
-            // SQLFabric changes END
           } finally {
             // bug 32589, post update may throw an exception if exception occurs
             // for any recipients
@@ -2985,23 +2825,11 @@ public abstract class AbstractRegionMap implements RegionMap {
     return true;
   }
 
-  // Asif: If the new value is an instance of SerializableDelta, then
-  // the old value requirement is a must & it needs to be faulted in
-  // if overflown to disk without affecting LRU? This is needed for
-  // Sql Fabric.
-  // [sumedh] store both the value in VM and the value in VM or disk;
-  // the former is used for updating the VM size calculations, while
-  // the latter is used in other places like passing to
-  // SqlfIndexManager or setting the old value in the event; this is
-  // required since using the latter for updating the size
-  // calculations will be incorrect in case the value was read from
-  // disk but not brought into the VM like what getValueInVMOrDisk
-  // method does when value is not found in VM
   // PRECONDITION: caller must be synced on re
   private void setOldValueInEvent(EntryEventImpl event, RegionEntry re, boolean cacheWrite, boolean requireOldValue) {
-    boolean needToSetOldValue = getIndexUpdater() != null || cacheWrite || requireOldValue || event.getOperation().guaranteesOldValue();
+    boolean needToSetOldValue = cacheWrite || requireOldValue || event.getOperation().guaranteesOldValue();
     if (needToSetOldValue) {
-      if (event.hasDelta() || event.getOperation().guaranteesOldValue()) {
+      if (event.getOperation().guaranteesOldValue()) {
         // In these cases we want to even get the old value from disk if it is not in memory
         ReferenceCountHelper.skipRefCountTracking();
         @Released Object oldValueInVMOrDisk = re.getValueOffHeapOrDiskWithoutFaultIn(event.getLocalRegion());
@@ -3186,7 +3014,6 @@ public abstract class AbstractRegionMap implements RegionMap {
     final boolean isClientTXOriginator = owner.cache.isClient() && !hasRemoteOrigin;
     final boolean isRegionReady = owner.isInitialized();
     @Released EntryEventImpl cbEvent = null;
-    @Released EntryEventImpl sqlfEvent = null;
     boolean invokeCallbacks = shouldCreateCBEvent(owner, isRegionReady);
     boolean cbEventInPending = false;
     cbEvent = createCBEvent(owner, putOp, key, newValue, txId, 
@@ -3202,12 +3029,6 @@ public abstract class AbstractRegionMap implements RegionMap {
       txHandleWANEvent(owner, cbEvent, txEntryState);
     }
     
-    if (/*owner.isUsedForPartitionedRegionBucket() && */ 
-       (getIndexUpdater() != null ||
-       (newValue instanceof com.gemstone.gemfire.internal.cache.delta.Delta))) {
-      sqlfEvent = createCBEvent(owner, putOp, key, newValue, txId, 
-          txEvent, eventId, aCallbackArgument,filterRoutingInfo,bridgeContext, txEntryState, versionTag, tailKey);
-    }
     boolean opCompleted = false;
     // Fix for Bug #44431. We do NOT want to update the region and wait
     // later for index INIT as region.clear() can cause inconsistency if
@@ -3240,9 +3061,6 @@ public abstract class AbstractRegionMap implements RegionMap {
                     cbEvent.setRegionEntry(re);
                     cbEvent.setOldValue(re.getValueInVM(owner)); // OFFHEAP eei
                   }
-                  if (sqlfEvent != null) {
-                    sqlfEvent.setOldValue(re.getValueInVM(owner)); // OFFHEAP eei
-                  }
 
                   boolean clearOccured = false;
                   // Set RegionEntry updateInProgress
@@ -3259,14 +3077,8 @@ public abstract class AbstractRegionMap implements RegionMap {
                     }
                     re.setValueResultOfSearch(putOp.isNetSearch());
                     try {
-                      // Rahul: applies the delta and sets the new value in 
-                      // region entry (required for sqlfabric delta).
                       processAndGenerateTXVersionTag(owner, cbEvent, re, txEntryState);
-                      if (newValue instanceof com.gemstone.gemfire.internal.cache.delta.Delta 
-                          && sqlfEvent != null) {
-                        //cbEvent.putExistingEntry(owner, re);
-                        sqlfEvent.putExistingEntry(owner, re);
-                      } else {
+                      {
                         re.setValue(owner, re.prepareValueForCache(owner, newValue, cbEvent, !putOp.isCreate()));
                       }
                       if (putOp.isCreate()) {
@@ -3276,9 +3088,7 @@ public abstract class AbstractRegionMap implements RegionMap {
                         // an issue with normal GFE Delta and will have to be fixed 
                         // in a similar manner and may be this fix the the one for 
                         // other delta can be combined.
-                        if (sqlfEvent != null) {
-                          owner.updateSizeOnPut(key, oldSize, sqlfEvent.getNewValueBucketSize());
-                        } else {
+                        {
                           owner.updateSizeOnPut(key, oldSize, owner.calculateRegionEntryValueSize(re));
                         }
                       }
@@ -3350,9 +3160,6 @@ public abstract class AbstractRegionMap implements RegionMap {
                     cbEvent.setRegionEntry(oldRe);
                     cbEvent.setOldValue(oldRe.getValueInVM(owner)); // OFFHEAP eei
                   }
-                  if (sqlfEvent != null) {
-                    sqlfEvent.setOldValue(oldRe.getValueInVM(owner)); // OFFHEAP eei
-                  }
                   boolean clearOccured = false;
                   // Set RegionEntry updateInProgress
                   if (owner.indexMaintenanceSynchronous) {
@@ -3370,11 +3177,7 @@ public abstract class AbstractRegionMap implements RegionMap {
                     try {
                       processAndGenerateTXVersionTag(owner, cbEvent, oldRe, txEntryState);
                       boolean wasTombstone = oldRe.isTombstone();
-                      if (newValue instanceof com.gemstone.gemfire.internal.cache.delta.Delta 
-                          && sqlfEvent != null ) {
-                        //cbEvent.putExistingEntry(owner, oldRe);
-                        sqlfEvent.putExistingEntry(owner, oldRe);
-                      } else {
+                      {
                         oldRe.setValue(owner, oldRe.prepareValueForCache(owner, newValue, cbEvent, !putOp.isCreate()));
                         if (wasTombstone) {
                           owner.unscheduleTombstone(oldRe);
@@ -3387,9 +3190,7 @@ public abstract class AbstractRegionMap implements RegionMap {
                         // an issue with normal GFE Delta and will have to be fixed 
                         // in a similar manner and may be this fix the the one for 
                         // other delta can be combined.
-                        if (sqlfEvent != null) {
-                          owner.updateSizeOnPut(key, oldSize, sqlfEvent.getNewValueBucketSize());
-                        } else {
+                        {
                           owner.updateSizeOnPut(key, oldSize, owner.calculateRegionEntryValueSize(oldRe));
                         }
                       }
@@ -3452,9 +3253,7 @@ public abstract class AbstractRegionMap implements RegionMap {
                 try {
                   
                   processAndGenerateTXVersionTag(owner, cbEvent, newRe, txEntryState);
-                  if (sqlfEvent != null ) {
-                    sqlfEvent.putNewEntry(owner,newRe);
-                  } else {
+                  {
                     newRe.setValue(owner, newRe.prepareValueForCache(owner, newValue, cbEvent, !putOp.isCreate()));
                   }
                   owner.updateSizeOnCreate(newRe.getKey(), owner.calculateRegionEntryValueSize(newRe));
@@ -3514,7 +3313,6 @@ public abstract class AbstractRegionMap implements RegionMap {
     }
     } finally {
       if (!cbEventInPending) cbEvent.release();
-      if (sqlfEvent != null) sqlfEvent.release();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketAdvisor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketAdvisor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketAdvisor.java
index f8e04a6..d085c52 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketAdvisor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketAdvisor.java
@@ -192,7 +192,6 @@ public class BucketAdvisor extends CacheDistributionAdvisor  {
     return advisor;
   }
 
-  // For SQLFabric ALTER TABLE that may change colocation
   public void resetParentAdvisor(int bucketId) {
     PartitionedRegion colocatedRegion = ColocationHelper
         .getColocatedRegion(this.pRegion);
@@ -1117,11 +1116,6 @@ public class BucketAdvisor extends CacheDistributionAdvisor  {
         // only one thread should be attempting to volunteer at one time
         return;
       }
-      // if member is still not initialized then don't volunteer for primary
-      final GemFireCacheImpl cache = (GemFireCacheImpl)getBucket().getCache();
-      if (!cache.doVolunteerForPrimary(this)) {
-        return;
-      }
       if (this.volunteeringDelegate == null) {
         this.volunteeringDelegate = new VolunteeringDelegate();
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java
index c87cc3d..e0f6fa2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/BucketRegion.java
@@ -19,7 +19,6 @@ package com.gemstone.gemfire.internal.cache;
 import com.gemstone.gemfire.*;
 import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.cache.partition.PartitionListener;
-import com.gemstone.gemfire.cache.query.internal.IndexUpdater;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.AtomicLongWithTerminalState;
@@ -34,8 +33,15 @@ import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.BucketAdvisor.BucketProfile;
 import com.gemstone.gemfire.internal.cache.FilterRoutingInfo.FilterInfo;
 import com.gemstone.gemfire.internal.cache.control.MemoryEvent;
-import com.gemstone.gemfire.internal.cache.delta.Delta;
-import com.gemstone.gemfire.internal.cache.partitioned.*;
+import com.gemstone.gemfire.internal.cache.partitioned.Bucket;
+import com.gemstone.gemfire.internal.cache.partitioned.DestroyMessage;
+import com.gemstone.gemfire.internal.cache.partitioned.InvalidateMessage;
+import com.gemstone.gemfire.internal.cache.partitioned.LockObject;
+import com.gemstone.gemfire.internal.cache.partitioned.PRTombstoneMessage;
+import com.gemstone.gemfire.internal.cache.partitioned.PartitionMessage;
+import com.gemstone.gemfire.internal.cache.partitioned.PutAllPRMessage;
+import com.gemstone.gemfire.internal.cache.partitioned.PutMessage;
+import com.gemstone.gemfire.internal.cache.partitioned.RemoveAllPRMessage;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientNotifier;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientTombstoneMessage;
@@ -1470,7 +1476,6 @@ implements Bucket
     .append("[path='").append(getFullPath())
     .append(";serial=").append(getSerialNumber())
     .append(";primary=").append(getBucketAdvisor().getProxyBucketRegion().isPrimary())
-    .append(";indexUpdater=").append(getIndexUpdater())
     .append("]")
     .toString();
   }
@@ -1695,10 +1700,8 @@ implements Bucket
       setDeltaIfNeeded(event);
     }
     if (msg != null) {
-      // The primary bucket member which is being modified remotely by a GemFire
+      // The primary bucket member which is being modified remotely by a
       // thread via a received PartitionedMessage
-      //Asif: Some of the adjunct recepients include those members which 
-      // are sqlFabricHub & would need old value along with news
       msg = msg.getMessageForRelayToListeners(event, adjunctRecipients);
       msg.setSender(this.partitionedRegion.getDistributionManager()
           .getDistributionManagerId());
@@ -1987,28 +1990,6 @@ implements Bucket
   public CacheWriter basicGetWriter() {
     return this.partitionedRegion.basicGetWriter();
   }
-   @Override
-  void cleanUpOnIncompleteOp(EntryEventImpl event,   RegionEntry re, 
-      boolean eventRecorded, boolean updateStats, boolean isReplace) {
-     
-    
-    if(!eventRecorded || isReplace) {
-      //No indexes updated so safe to remove.
-      this.entries.removeEntry(event.getKey(), re, updateStats) ;      
-    }/*else {
-      //if event recorded is true, that means as per event tracker entry is in
-      //system. As per sqlfabric, indexes have been updated. What is not done
-      // is basicPutPart2( distribution etc). So we do nothing as PR's re-attempt
-      // will do the required basicPutPart2. If we remove the entry here, than 
-      //event tracker will not allow re insertion. So either we do nothing or
-      //if we remove ,than we have to update sqlfindexes as well as undo recording
-      // of event.
-       //TODO:OQL indexes? : Hope they get updated during retry. The issue is that oql indexes
-       // get updated after distribute , so it is entirely possible that oql index are 
-        // not updated. what if retry fails?
-       
-    }*/
-  }
 
   /* (non-Javadoc)
    * @see com.gemstone.gemfire.internal.cache.partitioned.Bucket#getBucketOwners()
@@ -2058,7 +2039,7 @@ implements Bucket
       return 0;
     }
     if (!(value instanceof byte[]) && !(value instanceof CachedDeserializable)
-        && !(value instanceof com.gemstone.gemfire.Delta) && !(value instanceof Delta)
+        && !(value instanceof com.gemstone.gemfire.Delta)
         && !(value instanceof GatewaySenderEventImpl)) {
     // ezoerner:20090401 it's possible this value is a Delta
       throw new InternalGemFireError("DEBUG: calcMemSize: weird value (class " 
@@ -2198,10 +2179,6 @@ implements Bucket
   
 
   public void preDestroyBucket(int bucketId) {
-    final IndexUpdater indexUpdater = getIndexUpdater();
-    if (indexUpdater != null) {
-      indexUpdater.clearIndexes(this, bucketId);
-    }
   }
   @Override
   public void cleanupFailedInitialization()


[4/5] incubator-geode git commit: GEODE-1464: remove sqlf code

Posted by ds...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheDistributionAdvisor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheDistributionAdvisor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheDistributionAdvisor.java
index c4a8e27..fff9bab 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheDistributionAdvisor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheDistributionAdvisor.java
@@ -87,7 +87,7 @@ public class CacheDistributionAdvisor extends DistributionAdvisor  {
   protected static final int REQUIRES_NOTIFICATION_MASK = 0x8000;
   private static final int HAS_CACHE_SERVER_MASK = 0x10000;
   private static final int REQUIRES_OLD_VALUE_MASK = 0x20000;
-  private static final int MEMBER_UNINITIALIZED_MASK = 0x40000;
+  // unused 0x40000;
   private static final int PERSISTENCE_INITIALIZED_MASK = 0x80000;
   //Important below mentioned bit masks are not available 
   /**
@@ -229,8 +229,7 @@ public class CacheDistributionAdvisor extends DistributionAdvisor  {
         CacheProfile prof = (CacheProfile)profile;
 
         // if region in cache is not yet initialized, exclude
-        if (!prof.regionInitialized          // fix for bug 41102
-            || prof.memberUnInitialized) {
+        if (!prof.regionInitialized) { // fix for bug 41102
           return false;
         }
 
@@ -269,10 +268,6 @@ public class CacheDistributionAdvisor extends DistributionAdvisor  {
           if (!cp.regionInitialized) {
             return false;
           }
-          // if member is not yet initialized, exclude
-          if (cp.memberUnInitialized) {
-            return false;
-          }
           if (!cp.cachedOrAllEventsWithListener()) {
             return false;
           }
@@ -327,8 +322,7 @@ public class CacheDistributionAdvisor extends DistributionAdvisor  {
       public boolean include(Profile profile) {
         assert profile instanceof CacheProfile;
         CacheProfile cp = (CacheProfile)profile;
-        if (cp.dataPolicy.withReplication() && cp.regionInitialized
-            && !cp.memberUnInitialized) {
+        if (cp.dataPolicy.withReplication() && cp.regionInitialized) {
           return true;
         }
         return false;
@@ -350,10 +344,6 @@ public class CacheDistributionAdvisor extends DistributionAdvisor  {
         if (!cp.regionInitialized) {
           return false;
         }
-        // if member is not yet initialized, exclude
-        if (cp.memberUnInitialized) {
-          return false;
-        }
         DataPolicy dp = cp.dataPolicy;
         return dp.withStorage();
       }
@@ -544,14 +534,6 @@ public class CacheDistributionAdvisor extends DistributionAdvisor  {
      */
     public boolean regionInitialized;
 
-    /**
-     * True when member is still not ready to receive cache operations. Note
-     * that {@link #regionInitialized} may be still true so other members can
-     * proceed with GII etc. Currently used by SQLFabric to indicate that DDL
-     * replay is in progress and so cache operations/functions should not be
-     * routed to that node.
-     */
-    public boolean memberUnInitialized = false;
     
     /**
      * True when a members persistent store is initialized. Note that
@@ -615,7 +597,6 @@ public class CacheDistributionAdvisor extends DistributionAdvisor  {
       if (this.isGatewayEnabled) s |= IS_GATEWAY_ENABLED_MASK;
       if (this.isPersistent) s |= PERSISTENT_MASK;
       if (this.regionInitialized) s|= REGION_INITIALIZED_MASK;
-      if (this.memberUnInitialized) s |= MEMBER_UNINITIALIZED_MASK;
       if (this.persistentID != null) s|= PERSISTENT_ID_MASK;
       if (this.hasCacheServer) s|= HAS_CACHE_SERVER_MASK;
       if (this.requiresOldValueInEvents) s|= REQUIRES_OLD_VALUE_MASK;
@@ -693,7 +674,6 @@ public class CacheDistributionAdvisor extends DistributionAdvisor  {
       this.isGatewayEnabled = (s & IS_GATEWAY_ENABLED_MASK) != 0;
       this.isPersistent = (s & PERSISTENT_MASK) != 0;
       this.regionInitialized = ( (s & REGION_INITIALIZED_MASK) != 0 );
-      this.memberUnInitialized = (s & MEMBER_UNINITIALIZED_MASK) != 0;
       this.hasCacheServer = ( (s & HAS_CACHE_SERVER_MASK) != 0 );
       this.requiresOldValueInEvents = ((s & REQUIRES_OLD_VALUE_MASK) != 0);
       this.persistenceInitialized = (s & PERSISTENCE_INITIALIZED_MASK) != 0;
@@ -890,8 +870,6 @@ public class CacheDistributionAdvisor extends DistributionAdvisor  {
       sb.append("; scope=" + this.scope);
       sb.append("; regionInitialized=").append(
           String.valueOf(this.regionInitialized));
-      sb.append("; memberUnInitialized=").append(
-          String.valueOf(this.memberUnInitialized));
       sb.append("; inRecovery=" + this.inRecovery);
       sb.append("; subcription=" + this.subscriptionAttributes);
       sb.append("; isPartitioned=" + this.isPartitioned);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerLauncher.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerLauncher.java
index bb595d1..e982e32 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerLauncher.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerLauncher.java
@@ -362,7 +362,7 @@ public class CacheServerLauncher  {
     }
 
     // -J-Djava.awt.headless=true has been added for Mac platform where it
-    // causes an icon to appear for sqlf launched procs
+    // causes an icon to appear for launched procs
     // TODO: check which library/GemFire code causes awt to be touched
     vmArgs.add("-Djava.awt.headless=true");
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CachedDeserializableFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CachedDeserializableFactory.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CachedDeserializableFactory.java
index 83b0a58..ae60056 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CachedDeserializableFactory.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CachedDeserializableFactory.java
@@ -39,13 +39,6 @@ public class CachedDeserializableFactory {
   public static boolean STORE_ALL_VALUE_FORMS = Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "STORE_ALL_VALUE_FORMS");
 
   /**
-   * Currently GFE always wants a CachedDeserializable wrapper.
-   */
-  public static final boolean preferObject() {
-    return false;
-  }
-  
-  /**
    * Creates and returns an instance of CachedDeserializable that contains the
    * specified byte array.
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ColocationHelper.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ColocationHelper.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ColocationHelper.java
index b53ed31..012a77f 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ColocationHelper.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ColocationHelper.java
@@ -47,61 +47,7 @@ public class ColocationHelper {
 
   /** Logging mechanism for debugging */
   private static final Logger logger = LogService.getLogger();
-   /**
-    * An utility method to retrieve colocated region name of a given partitioned
-    * region without waiting on initialize
-    *
-    * @param partitionedRegion
-    * @return colocated PartitionedRegion
-    * @since GemFire cheetah
-    */
-  public static PartitionedRegion getColocatedRegionName(
-      final PartitionedRegion partitionedRegion) {
-    Assert.assertTrue(partitionedRegion != null); // precondition1
-    String colocatedWith = partitionedRegion.getPartitionAttributes().getColocatedWith();
-    if (colocatedWith == null) {
-      // the region is not colocated with any region
-      return null;
-    }
-    PartitionedRegion colocatedPR = partitionedRegion.getColocatedWithRegion();
-    if (colocatedPR != null && !colocatedPR.isLocallyDestroyed
-        && !colocatedPR.isDestroyed()) {
-      return colocatedPR;
-    }
-    Region prRoot = PartitionedRegionHelper.getPRRoot(partitionedRegion
-        .getCache());
-    PartitionRegionConfig prConf = (PartitionRegionConfig)prRoot
-        .get(getRegionIdentifier(colocatedWith));
-    int prID = -1; 
-    try {
-      if (prConf == null) {
-        colocatedPR = getColocatedPR(partitionedRegion, colocatedWith);
-      }
-      else {
-        prID = prConf.getPRId();
-        colocatedPR = PartitionedRegion.getPRFromId(prID);
-        if (colocatedPR == null && prID > 0) {
-          // colocatedPR might have not called registerPartitionedRegion() yet, but since prID is valid,
-          // we are able to get colocatedPR and do colocatedPR.waitOnBucketMetadataInitialization()
-          colocatedPR = getColocatedPR(partitionedRegion, colocatedWith);
-        }
-      }
-    }
-    catch (PRLocallyDestroyedException e) {
-      if (logger.isDebugEnabled()) {
-        logger.debug("PRLocallyDestroyedException : Region with prId=" + prID
-            + " is locally destroyed on this node", e);
-      } 
-    } 
-    return colocatedPR;
-  }
-    private static PartitionedRegion getColocatedPR(
-      final PartitionedRegion partitionedRegion, final String colocatedWith) {
-    PartitionedRegion colocatedPR = (PartitionedRegion) partitionedRegion
-        .getCache().getPartitionedRegion(colocatedWith, false);
-    assert colocatedPR != null;
-    return colocatedPR;
-  }
+
   /** Whether to ignore missing parallel queues on restart
    * if they are not attached to the region. See bug 50120. Mutable
    * for tests.
@@ -517,17 +463,6 @@ public class ColocationHelper {
     
     return prRegion;
   }
-  
-  // Gemfirexd will skip initialization for PR, so just get region name without waitOnInitialize
-  public static PartitionedRegion getLeaderRegionName(PartitionedRegion prRegion) {
-    PartitionedRegion parentRegion;
-    
-    while((parentRegion = getColocatedRegionName(prRegion)) != null) {
-      prRegion = parentRegion;
-    } 
-      
-    return prRegion;
-  }
 
   private static String getRegionIdentifier(String regionName) {
     if (regionName.startsWith("/")) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DestroyOperation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DestroyOperation.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DestroyOperation.java
index e267190..5bfb3cc 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DestroyOperation.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DestroyOperation.java
@@ -129,9 +129,6 @@ public class DestroyOperation extends DistributedCacheOperation
     @Retained
     protected final InternalCacheEvent createEvent(DistributedRegion rgn)
         throws EntryNotFoundException {
-      if (rgn.keyRequiresRegionContext()) {
-        ((KeyWithRegionContext)this.key).setRegionContext(rgn);
-      }
       EntryEventImpl ev = createEntryEvent(rgn);
       boolean evReturned = false;
       try {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java
index e015460..5da0d9a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskEntry.java
@@ -85,13 +85,6 @@ public interface DiskEntry extends RegionEntry {
    * @param context
    */
   public void handleValueOverflow(RegionEntryContext context);
-  
-  /**
-   * In some cases we need to do something just after we unset the value
-   * from a DiskEntry that has been moved (i.e. overflowed) to disk.
-   * @param context
-   */
-  public void afterValueOverflow(RegionEntryContext context);
 
   /**
    * Returns true if the DiskEntry value is equal to {@link Token#DESTROYED}, {@link Token#REMOVED_PHASE1}, or {@link Token#REMOVED_PHASE2}.
@@ -247,27 +240,6 @@ public interface DiskEntry extends RegionEntry {
         }
       }
     }
-      
-    /**
-     * Returns false if the entry is INVALID (or LOCAL_INVALID). Determines this
-     * without faulting in the value from disk.
-     * 
-     * @since GemFire 3.2.1
-     */
-    /* TODO prpersist - Do we need this method? It was added by the sqlf merge
-    static boolean isValid(DiskEntry entry, DiskRegion dr) {
-      synchronized (entry) {
-        if (entry.isRecovered()) {
-          // We have a recovered entry whose value is still on disk.
-          // So take a peek at it without faulting it in.
-          //long id = entry.getDiskId().getKeyId();
-          //entry.getDiskId().setKeyId(-id);
-          byte bits = dr.getBits(entry.getDiskId());
-          //TODO Asif:Check if resetting is needed
-          return !EntryBits.isInvalid(bits) && !EntryBits.isLocalInvalid(bits);
-        }
-      }
-    }*/
 
     static boolean isOverflowedToDisk(DiskEntry de, DiskRegion dr, DistributedRegion.DiskPosition dp,RegionEntryContext context) {
       Object v = null;
@@ -372,10 +344,6 @@ public interface DiskEntry extends RegionEntry {
           dr.releaseReadLock();
         }
       }
-      final boolean isEagerDeserialize = entry.isEagerDeserialize();
-      if (isEagerDeserialize) {
-        entry.clearEagerDeserialize();
-      }
       if (Token.isRemovedFromDisk(v)) {
         // fix for bug 31757
         return false;
@@ -386,30 +354,15 @@ public interface DiskEntry extends RegionEntry {
             entry.setSerialized(false);
             entry.value = cd.getDeserializedForReading();
             
-            //For SQLFire we prefer eager deserialized
-//            if(v instanceof ByteSource) {
-//              entry.setEagerDeserialize();
-//            }
           } else {
             // don't serialize here if it is not already serialized
             
             Object tmp = cd.getValue();
-          //For SQLFire we prefer eager deserialized
-//            if(v instanceof ByteSource) {
-//              entry.setEagerDeserialize();
-//            }
             if (tmp instanceof byte[]) {
               byte[] bb = (byte[])tmp;
               entry.value = bb;
               entry.setSerialized(true);
             }
-            else if (isEagerDeserialize && tmp instanceof byte[][]) {
-              // optimize for byte[][] since it will need to be eagerly deserialized
-              // for SQLFabric
-              entry.value = tmp;
-              entry.setEagerDeserialize();
-              entry.setSerialized(true);
-            }
             else {
               try {
                 HeapDataOutputStream hdos = new HeapDataOutputStream(Version.CURRENT);
@@ -437,12 +390,6 @@ public interface DiskEntry extends RegionEntry {
         entry.value = v;
         entry.setSerialized(false);
       }
-      else if (isEagerDeserialize && v instanceof byte[][]) {
-        // optimize for byte[][] since it will need to be eagerly deserialized
-        // for SQLFabric
-        entry.value = v;
-        entry.setEagerDeserialize();
-      }
       else if (v == Token.INVALID) {
         entry.setInvalid();
       }
@@ -460,11 +407,7 @@ public interface DiskEntry extends RegionEntry {
             return false;
           }
         }
-      if (CachedDeserializableFactory.preferObject()) {
-        entry.value = preparedValue;
-        entry.setEagerDeserialize();
-      }
-      else {
+      {
         try {
           HeapDataOutputStream hdos = new HeapDataOutputStream(Version.CURRENT);
           BlobHelper.serializeTo(preparedValue, hdos);
@@ -833,9 +776,7 @@ public interface DiskEntry extends RegionEntry {
         // to the file using the off-heap memory with no extra copying.
         // So we give preference to getRawNewValue over getCachedSerializedNewValue
         Object rawValue = null;
-        if (!event.hasDelta()) {
-          // We don't do this for the delta case because getRawNewValue returns delta
-          // and we want to write the entire new value to disk.
+        {
           rawValue = event.getRawNewValue();
           if (wrapOffHeapReference(rawValue)) {
             return new OffHeapValueWrapper((StoredObject) rawValue);
@@ -969,13 +910,8 @@ public interface DiskEntry extends RegionEntry {
           // Second, do the stats done for the current recovered value
           if (re.getRecoveredKeyId() < 0) {
             if (!entry.isValueNull()) {
-              try {
-                entry.handleValueOverflow(region);
-                entry.setValueWithContext(region, null); // fixes bug 41119
-              }finally {
-                entry.afterValueOverflow(region);
-              }
-              
+              entry.handleValueOverflow(region);
+              entry.setValueWithContext(region, null); // fixes bug 41119
             }
             dr.incNumOverflowOnDisk(1L);
             dr.incNumOverflowBytesOnDisk(did.getValueLength());
@@ -989,11 +925,7 @@ public interface DiskEntry extends RegionEntry {
         }
         else {
           //The new value in the entry needs to be set after the disk writing 
-          // has succeeded. If not , for GemFireXD , it is possible that other thread
-          // may pick this transient value from region entry ( which for 
-          //offheap will eventually be released ) as index key, 
-          //given that this operation is bound to fail in case of
-          //disk access exception.
+          // has succeeded.
           
           //entry.setValueWithContext(region, newValue); // OFFHEAP newValue already prepared
           
@@ -1008,10 +940,7 @@ public interface DiskEntry extends RegionEntry {
           if (dr.isBackup()) {
             dr.testIsRecoveredAndClear(did); // fixes bug 41409
             if (dr.isSync()) {
-              //In case of compression the value is being set first 
-              // because atleast for now , GemFireXD does not support compression
-              // if and when it does support, this needs to be taken care of else
-              // we risk Bug 48965
+              //In case of compression the value is being set first
               if (AbstractRegionEntry.isCompressible(dr, newValue)) {
                 entry.setValueWithContext(region, newValue); // OFFHEAP newValue already prepared
                 
@@ -1134,12 +1063,8 @@ public interface DiskEntry extends RegionEntry {
               false));
         } else {
           if (!oldValueWasNull) {
-            try {
-              entry.handleValueOverflow(context);
-              entry.setValueWithContext(context,null); // fixes bug 41119
-            }finally {
-              entry.afterValueOverflow(context);
-            }
+            entry.handleValueOverflow(context);
+            entry.setValueWithContext(context,null); // fixes bug 41119
           }
         }
         if (entry instanceof LRUEntry) {
@@ -1218,11 +1143,6 @@ public interface DiskEntry extends RegionEntry {
       boolean lruFaultedIn = false;
       boolean done = false;
       try {
-      //Asif: If the entry is instance of LRU then DidkRegion cannot be null.
-      //Since SqlFabric is accessing this method direcly & it passes the owning region,
-      //if the region happens to be persistent PR type, the owning region passed is PR,
-      // but it will have DiskRegion as null. SqlFabric takes care of passing owning region
-      // as BucketRegion in case of Overflow type entry. This is fix for Bug # 41804
       if ( entry instanceof LRUEntry && !dr.isSync() ) {
         synchronized (entry) {
           DiskId did = entry.getDiskId();
@@ -1391,10 +1311,8 @@ public interface DiskEntry extends RegionEntry {
      * Sets the value in the entry.
      * This is only called by the faultIn code once it has determined that
      * the value is no longer in memory.
-     * return the result will only be off-heap if the value is a sqlf ByteSource. Otherwise result will be on-heap.
      * Caller must have "entry" synced.
      */
-    @Retained
     private static Object readValueFromDisk(DiskEntry entry, DiskRecoveryStore region) {
 
       DiskRegionView dr = region.getDiskRegionView();
@@ -1407,16 +1325,8 @@ public interface DiskEntry extends RegionEntry {
       synchronized (did) {
         Object value = getValueFromDisk(dr, did, null);
         if (value == null) return null;
-        @Unretained Object preparedValue = setValueOnFaultIn(value, did, entry, dr, region);
-        // For Sqlfire we want to return the offheap representation.
-        // So we need to retain it for the caller to release.
-        /*if (preparedValue instanceof ByteSource) {
-          // This is the only case in which we return a retained off-heap ref.
-          ((ByteSource)preparedValue).retain();
-          return preparedValue;
-        } else */{
-          return value;
-        }
+        setValueOnFaultIn(value, did, entry, dr, region);
+        return value;
       }
       } finally {
         dr.releaseReadLock();
@@ -1464,16 +1374,7 @@ public interface DiskEntry extends RegionEntry {
 
     static Object readRawValue(byte[] valueBytes, Version version,
         ByteArrayDataInput in) {
-      /*
-      final StaticSystemCallbacks sysCb;
-      if (version != null && (sysCb = GemFireCacheImpl.FactoryStatics
-          .systemCallbacks) != null) {
-        // may need to change serialized shape for SQLFire
-        return sysCb.fromVersion(valueBytes, false, version, in);
-      }
-      else */ {
-        return valueBytes;
-      }
+      return valueBytes;
     }
 
     public static void incrementBucketStats(Object owner,
@@ -1521,12 +1422,6 @@ public interface DiskEntry extends RegionEntry {
         did = entry.getDiskId();
       }
       
-      // Notify the SQLFire IndexManager if present
-     /* final IndexUpdater indexUpdater = region.getIndexUpdater();
-      if(indexUpdater != null && dr.isSync()) {
-        indexUpdater.onOverflowToDisk(entry);
-      }*/
-      
       int change = 0;
       boolean scheduledAsyncHere = false;
       dr.acquireReadLock();
@@ -1561,13 +1456,8 @@ public interface DiskEntry extends RegionEntry {
           // do the stats when it is actually written to disk
         } else {
           region.updateSizeOnEvict(entry.getKey(), oldSize);
-          //did.setValueSerializedSize(byteSizeOnDisk);
-          try {
-            entry.handleValueOverflow(region);
-            entry.setValueWithContext(region,null);
-          }finally {
-            entry.afterValueOverflow(region);
-          }
+          entry.handleValueOverflow(region);
+          entry.setValueWithContext(region,null);
           movedValueToDisk = true;
           change = ((LRUClockNode)entry).updateEntrySize(ccHelper);
         }
@@ -1648,12 +1538,8 @@ public interface DiskEntry extends RegionEntry {
                 dr.incNumOverflowBytesOnDisk(did.getValueLength());
                 incrementBucketStats(region, 0/*InVM*/, 0/*OnDisk*/,
                                      did.getValueLength());
-                try {
-                  entry.handleValueOverflow(region);
-                  entry.setValueWithContext(region,null);
-                }finally {
-                  entry.afterValueOverflow(region);
-                }
+                entry.handleValueOverflow(region);
+                entry.setValueWithContext(region,null);
               }
               
               //See if we the entry we wrote to disk has the same tag
@@ -1769,12 +1655,8 @@ public interface DiskEntry extends RegionEntry {
                 dr.incNumOverflowBytesOnDisk(did.getValueLength());
                 incrementBucketStats(region, 0/*InVM*/, 0/*OnDisk*/,
                                      did.getValueLength());
-                try {
-                 entry.handleValueOverflow(region);
-                 entry.setValueWithContext(region,null);
-                }finally {
-                  entry.afterValueOverflow(region);
-                }
+                entry.handleValueOverflow(region);
+                entry.setValueWithContext(region,null);
               }
             } catch (RegionClearedException ignore) {
               // no need to do the op since it was clobbered by a region clear

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskInitFile.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskInitFile.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskInitFile.java
index 018a065..c6533a5 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskInitFile.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskInitFile.java
@@ -2435,8 +2435,6 @@ public class DiskInitFile implements DiskInitFileInterpreter {
 
   /**
    * Additional flags for a disk region that are persisted in its meta-data.
-   * Currently only few for GemFireXD added here but all other boolean flags also
-   * be better moved here.
    * 
    * @since GemFire 7.0
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskRegion.java
index a2092fd..3f511f0 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskRegion.java
@@ -791,8 +791,6 @@ public class DiskRegion extends AbstractDiskRegion {
           DiskId id = de.getDiskId();
           if (id != null) {
             synchronized (id) {
-              // SQLFabric: give a chance to copy key from value bytes when key
-              // is just a pointer to value row
               re.setValueToNull(); // TODO why call _setValue twice in a row?
               re.removePhase2();
               id.unmarkForWriting();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskWriteAttributesImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskWriteAttributesImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskWriteAttributesImpl.java
index 9bb148b..3e005cb 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskWriteAttributesImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DiskWriteAttributesImpl.java
@@ -83,18 +83,6 @@ public final class DiskWriteAttributesImpl implements DiskWriteAttributes
   public static final String SYNCHRONOUS_PROPERTY = "synchronous";
 
   /**
-   * The property used to specify the base directory for Sql Fabric persistence
-   * of Gateway Queues, Tables, Data Dictionary etc.
-   */
-  public static final String SYS_PERSISTENT_DIR = "sys-disk-dir";
-
-  /**
-   * The system property for {@link #SYS_PERSISTENT_DIR}.
-   */
-  public static final String SYS_PERSISTENT_DIR_PROP = "sqlfabric."
-      + SYS_PERSISTENT_DIR;
-
-  /**
    * Default disk directory size in megabytes
    * 
    * @since GemFire 5.1
@@ -477,83 +465,4 @@ public final class DiskWriteAttributesImpl implements DiskWriteAttributes
   {
     return DEFAULT_SYNC_DWA;
   }
-
-
-  // Asif: Sql Fabric related helper methods.
-  // These static functions need to be moved to a better place.
-  // preferably in sql Fabric source tree but since GatewayImpl is also
-  // utilizing it, we have no option but to keep it here.
-  public static String generateOverFlowDirName(String dirName) {
-    dirName = generatePersistentDirName(dirName);
-    final GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
-    if (cache == null) {
-      throw new CacheClosedException(
-          "DiskWriteAttributesImpl::generateOverFlowDirName: no cache found.");
-    }
-    /* [sumedh] no need of below since sys-disk-dir is VM specific anyways
-    char fileSeparator = System.getProperty("file.separator").charAt(0);
-    DistributedMember member = cache.getDistributedSystem()
-        .getDistributedMember();
-    String host = member.getHost();
-    int pid = member.getProcessId();
-    final StringBuilder temp = new StringBuilder(dirName);
-    temp.append(fileSeparator);
-    temp.append(host);
-    temp.append('-');
-    temp.append(pid);
-    return temp.toString();
-    */
-    return dirName;
-  }
-
-  public static String generatePersistentDirName(String dirPath) {
-    String baseDir = System.getProperty(SYS_PERSISTENT_DIR_PROP);
-    if (baseDir == null) {
-    //Kishor : TODO : Removing old wan related code
-      //baseDir = GatewayQueueAttributes.DEFAULT_OVERFLOW_DIRECTORY;
-      baseDir = ".";
-    }
-    if (dirPath != null) {
-      File dirProvided = new File(dirPath);
-      // Is the directory path absolute?
-      // For Windows this will check for drive letter. However, we want
-      // to allow for no drive letter so prepend the drive.
-      boolean isAbsolute = dirProvided.isAbsolute();
-      if (!isAbsolute) {
-        String driveName;
-        // get the current drive for Windows and prepend
-        if ((dirPath.charAt(0) == '/' || dirPath.charAt(0) == '\\')
-            && (driveName = getCurrentDriveName()) != null) {
-          isAbsolute = true;
-          dirPath = driveName + dirPath;
-        }
-      }
-      if (!isAbsolute) {
-        // relative path so resolve it relative to parent dir
-        dirPath = new File(baseDir, dirPath).getAbsolutePath();
-      }
-    }
-    else {
-      dirPath = baseDir;
-    }
-    return dirPath;
-  }
-
-  /**
-   * Get the drive name of current working directory for windows else return
-   * null for non-Windows platform (somewhat of a hack -- see if something
-   * cleaner can be done for this).
-   */
-  public static String getCurrentDriveName() {
-    if (System.getProperty("os.name").startsWith("Windows")) {
-      try {
-        // get the current drive
-        return new File(".").getCanonicalPath().substring(0, 2);
-      } catch (IOException ex) {
-        throw new IllegalArgumentException(
-            "Failed in setting the overflow directory", ex);
-      }
-    }
-    return null;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXState.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXState.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXState.java
index cafdb80..ed59108 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXState.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXState.java
@@ -447,7 +447,7 @@ public class DistTXState extends TXState {
         postPutAll(dtop.getPutAllOperation(), versions, dtop.region);
       } else {
         result = putEntryOnRemote(dtop, false/* ifNew */,
-          dtop.hasDelta()/* ifOld */, null/* expectedOldValue */,
+          false/* ifOld */, null/* expectedOldValue */,
           false/* requireOldValue */, 0L/* lastModified */, true/*
                                                                  * overwriteDestroyed
                                                                  * *not*
@@ -572,7 +572,7 @@ public class DistTXState extends TXState {
           @Released EntryEventImpl ev = PutAllPRMessage.getEventFromEntry(theRegion,
               myId, myId, i, putallOp.putAllData, false, putallOp
                   .getBaseEvent().getContext(), false, !putallOp.getBaseEvent()
-                  .isGenerateCallbacks(), false);
+                  .isGenerateCallbacks());
           try {
 //            ev.setPutAllOperation(putallOp);
             

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXStateOnCoordinator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXStateOnCoordinator.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXStateOnCoordinator.java
index 33bec1c..436c637 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXStateOnCoordinator.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXStateOnCoordinator.java
@@ -359,7 +359,7 @@ public final class DistTXStateOnCoordinator extends DistTXState implements
         postPutAll(dtop.getPutAllOperation(), versions, dtop.region);
       } else {
         result = putEntry(dtop, false/* ifNew */,
-          dtop.hasDelta()/* ifOld */, null/* expectedOldValue */,
+          false/* ifOld */, null/* expectedOldValue */,
           false/* requireOldValue */, 0L/* lastModified */, true/*
                                                                  * overwriteDestroyed
                                                                  * *not*

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java
index 6b1073b..83d4c5a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedCacheOperation.java
@@ -101,13 +101,6 @@ public abstract class DistributedCacheOperation {
   public static final byte DESERIALIZATION_POLICY_NONE = (byte)0;
 
   /**
-   * Deserialization policy: deserialize eagerly (for Deltas)
-   * 
-   * @since GemFire 5.7
-   */
-  public static final byte DESERIALIZATION_POLICY_EAGER = (byte)1;
-
-  /**
    * Deserialization policy: deserialize lazily (for all other objects)
    * 
    * @since GemFire 5.7
@@ -115,14 +108,11 @@ public abstract class DistributedCacheOperation {
   public static final byte DESERIALIZATION_POLICY_LAZY = (byte)2;
   
   /**
-   * @param deserializationPolicy must be one of the following: DESERIALIZATION_POLICY_NONE, DESERIALIZATION_POLICY_EAGER, DESERIALIZATION_POLICY_LAZY.
+   * @param deserializationPolicy must be one of the following: DESERIALIZATION_POLICY_NONE, DESERIALIZATION_POLICY_LAZY.
    */
   public static void writeValue(final byte deserializationPolicy, final Object vObj, final byte[] vBytes, final DataOutput out) throws IOException {
     if (vObj != null) {
-      if (deserializationPolicy == DESERIALIZATION_POLICY_EAGER) {
-        // for DESERIALIZATION_POLICY_EAGER avoid extra byte array serialization
-        DataSerializer.writeObject(vObj, out);
-      } else if (deserializationPolicy == DESERIALIZATION_POLICY_NONE) {
+      if (deserializationPolicy == DESERIALIZATION_POLICY_NONE) {
         // We only have NONE with a vObj when vObj is off-heap and not serialized.
         StoredObject so = (StoredObject) vObj;
         assert !so.isSerialized();
@@ -131,14 +121,7 @@ public abstract class DistributedCacheOperation {
         DataSerializer.writeObjectAsByteArray(vObj, out);
       }
     } else {
-      if (deserializationPolicy == DESERIALIZATION_POLICY_EAGER) {
-        // object is already in serialized form in the byte array.
-        // So just write the bytes to the stream.
-        // fromData will call readObject which will deserialize to object form.
-        out.write(vBytes);
-      } else {
-        DataSerializer.writeByteArray(vBytes, out);
-      }
+      DataSerializer.writeByteArray(vBytes, out);
     }    
   }
   // static values for oldValueIsObject
@@ -151,7 +134,6 @@ public abstract class DistributedCacheOperation {
    */
   public static byte valueIsToDeserializationPolicy(boolean oldValueIsSerialized) {
     if (!oldValueIsSerialized) return DESERIALIZATION_POLICY_NONE;
-    if (CachedDeserializableFactory.preferObject()) return DESERIALIZATION_POLICY_EAGER;
     return DESERIALIZATION_POLICY_LAZY;
   }
 
@@ -180,8 +162,6 @@ public abstract class DistributedCacheOperation {
     switch (policy) {
     case DESERIALIZATION_POLICY_NONE:
       return "NONE";
-    case DESERIALIZATION_POLICY_EAGER:
-      return "EAGER";
     case DESERIALIZATION_POLICY_LAZY:
       return "LAZY";
     default:
@@ -863,8 +843,6 @@ public abstract class DistributedCacheOperation {
 
     private final static int INHIBIT_NOTIFICATIONS_MASK = 0x400;
 
-    protected final static short IS_PUT_DML = 0x100;
-
     public boolean needsRouting;
 
     protected String regionPath;
@@ -1364,9 +1342,6 @@ public abstract class DistributedCacheOperation {
       }
       if ((extBits & INHIBIT_NOTIFICATIONS_MASK) != 0) {
         this.inhibitAllNotifications = true;
-	  if (this instanceof PutAllMessage) {
-        ((PutAllMessage) this).setPutDML((extBits & IS_PUT_DML) != 0);
-      }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedPutAllOperation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedPutAllOperation.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedPutAllOperation.java
index 5d71ef2..e73ca35 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedPutAllOperation.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedPutAllOperation.java
@@ -51,7 +51,6 @@ import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.FilterRoutingInfo.FilterInfo;
-import com.gemstone.gemfire.internal.cache.delta.Delta;
 import com.gemstone.gemfire.internal.cache.ha.ThreadIdentifier;
 import com.gemstone.gemfire.internal.cache.partitioned.PutAllPRMessage;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
@@ -416,8 +415,7 @@ public class DistributedPutAllOperation extends AbstractUpdateOperation
      * {@link PutAllPRMessage#toData(DataOutput)} <br>
      * {@link RemotePutAllMessage#toData(DataOutput)} <br>
      */
-    public final void toData(final DataOutput out, 
-        final boolean requiresRegionContext) throws IOException {
+    public final void toData(final DataOutput out) throws IOException {
       Object key = this.key;
       final Object v = this.value;
       DataSerializer.writeObject(key, out);
@@ -856,7 +854,6 @@ public class DistributedPutAllOperation extends AbstractUpdateOperation
     PutAllMessage msg = new PutAllMessage();
     msg.eventId = event.getEventId();
     msg.context = event.getContext();
-    msg.setPutDML(event.isPutDML());
     return msg;
   }
 
@@ -870,7 +867,7 @@ public class DistributedPutAllOperation extends AbstractUpdateOperation
   public PutAllPRMessage createPRMessagesNotifyOnly(int bucketId) {
     final EntryEventImpl event = getBaseEvent();
     PutAllPRMessage prMsg = new PutAllPRMessage(bucketId, putAllDataSize, true,
-        event.isPossibleDuplicate(), !event.isGenerateCallbacks(), event.getCallbackArgument(), false /*isPutDML*/);
+        event.isPossibleDuplicate(), !event.isGenerateCallbacks(), event.getCallbackArgument());
     if (event.getContext() != null) {
       prMsg.setBridgeContext(event.getContext());
     }
@@ -899,7 +896,7 @@ public class DistributedPutAllOperation extends AbstractUpdateOperation
       PutAllPRMessage prMsg = (PutAllPRMessage)prMsgMap.get(bucketId);
       if (prMsg == null) {
         prMsg = new PutAllPRMessage(bucketId.intValue(), putAllDataSize, false,
-            event.isPossibleDuplicate(), !event.isGenerateCallbacks(), event.getCallbackArgument(), event.isPutDML());
+            event.isPossibleDuplicate(), !event.isGenerateCallbacks(), event.getCallbackArgument());
         prMsg.setTransactionDistributed(event.getRegion().getCache().getTxManager().isDistributed());
 
         // set dpao's context(original sender) into each PutAllMsg
@@ -1076,8 +1073,6 @@ public class DistributedPutAllOperation extends AbstractUpdateOperation
 
     protected EventID eventId = null;
     
-    private transient boolean isPutDML = false;
-
     protected static final short HAS_BRIDGE_CONTEXT = UNRESERVED_FLAGS_START;
     protected static final short SKIP_CALLBACKS =
       (short)(HAS_BRIDGE_CONTEXT << 1);
@@ -1132,13 +1127,10 @@ public class DistributedPutAllOperation extends AbstractUpdateOperation
      * @param rgn
      *          the region the entry is put in
      */
-    public void doEntryPut(PutAllEntryData entry, DistributedRegion rgn,
-        boolean requiresRegionContext, boolean isPutDML) {
+    public void doEntryPut(PutAllEntryData entry, DistributedRegion rgn) {
       @Released EntryEventImpl ev = PutAllMessage.createEntryEvent(entry, getSender(), 
-          this.context, rgn,
-          requiresRegionContext, this.possibleDuplicate,
+          this.context, rgn, this.possibleDuplicate,
           this.needsRouting, this.callbackArg, true, skipCallbacks);
-      ev.setPutDML(isPutDML);
       // we don't need to set old value here, because the msg is from remote. local old value will get from next step
       try {
         super.basicOperateOnRegion(ev, rgn);
@@ -1158,7 +1150,6 @@ public class DistributedPutAllOperation extends AbstractUpdateOperation
      * @param sender
      * @param context
      * @param rgn
-     * @param requiresRegionContext
      * @param possibleDuplicate
      * @param needsRouting
      * @param callbackArg
@@ -1167,13 +1158,10 @@ public class DistributedPutAllOperation extends AbstractUpdateOperation
     @Retained
     public static EntryEventImpl createEntryEvent(PutAllEntryData entry,
         InternalDistributedMember sender, ClientProxyMembershipID context,
-        DistributedRegion rgn, boolean requiresRegionContext, 
+        DistributedRegion rgn,
         boolean possibleDuplicate, boolean needsRouting, Object callbackArg,
         boolean originRemote, boolean skipCallbacks) {
       final Object key = entry.getKey();
-      if (requiresRegionContext) {
-        ((KeyWithRegionContext)key).setRegionContext(rgn);
-      }
       EventID evId = entry.getEventID();
       @Retained EntryEventImpl ev = EntryEventImpl.create(rgn, entry.getOp(),
           key, null/* value */, callbackArg,
@@ -1225,14 +1213,13 @@ public class DistributedPutAllOperation extends AbstractUpdateOperation
       
       rgn.syncBulkOp(new Runnable() {
         public void run() {
-          final boolean requiresRegionContext = rgn.keyRequiresRegionContext();
           final boolean isDebugEnabled = logger.isDebugEnabled();
           for (int i = 0; i < putAllDataSize; ++i) {
             if (isDebugEnabled) {
               logger.debug("putAll processing {} with {} sender={}", putAllData[i], putAllData[i].versionTag, sender);
             }
             putAllData[i].setSender(sender);
-            doEntryPut(putAllData[i], rgn, requiresRegionContext,  isPutDML);
+            doEntryPut(putAllData[i], rgn);
           }
         }
       }, ev.getEventId());
@@ -1283,10 +1270,6 @@ public class DistributedPutAllOperation extends AbstractUpdateOperation
         EntryVersionsList versionTags = new EntryVersionsList(putAllDataSize);
 
         boolean hasTags = false;
-        // get the "keyRequiresRegionContext" flag from first element assuming
-        // all key objects to be uniform
-        final boolean requiresRegionContext =
-          (this.putAllData[0].key instanceof KeyWithRegionContext);
         for (int i = 0; i < this.putAllDataSize; i++) {
           if (!hasTags && putAllData[i].versionTag != null) {
             hasTags = true;
@@ -1294,7 +1277,7 @@ public class DistributedPutAllOperation extends AbstractUpdateOperation
           VersionTag<?> tag = putAllData[i].versionTag;
           versionTags.add(tag);
           putAllData[i].versionTag = null;
-          this.putAllData[i].toData(out, requiresRegionContext);
+          this.putAllData[i].toData(out);
           this.putAllData[i].versionTag = tag;
         }
 
@@ -1342,11 +1325,7 @@ public class DistributedPutAllOperation extends AbstractUpdateOperation
         Object valueObj = null;
         Object v = entry.getValue();
         byte deserializationPolicy;
-        if (v instanceof Delta) {
-          deserializationPolicy = DESERIALIZATION_POLICY_EAGER;
-          valueObj = v;
-        }
-        else if (v instanceof CachedDeserializable) {
+        if (v instanceof CachedDeserializable) {
           deserializationPolicy = DESERIALIZATION_POLICY_LAZY;
           valueBytes = ((CachedDeserializable)v).getSerializedValue();
         }
@@ -1360,18 +1339,5 @@ public class DistributedPutAllOperation extends AbstractUpdateOperation
       }
       return Arrays.asList(ops);
     }
-    
-    public void setPutDML(boolean val) {
-      this.isPutDML = val;
-    }
-    
-    @Override
-    protected short computeCompressedExtBits(short bits) {
-      bits = super.computeCompressedExtBits(bits);
-      if (isPutDML) {
-        bits |= IS_PUT_DML;
-      }
-      return bits;
-    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
index eed5268..b42a617 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegion.java
@@ -28,7 +28,6 @@ import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.execute.ResultCollector;
 import com.gemstone.gemfire.cache.execute.ResultSender;
 import com.gemstone.gemfire.cache.persistence.PersistentReplicatesOfflineException;
-import com.gemstone.gemfire.cache.query.internal.IndexUpdater;
 import com.gemstone.gemfire.cache.wan.GatewaySender;
 import com.gemstone.gemfire.distributed.DistributedLockService;
 import com.gemstone.gemfire.distributed.DistributedMember;
@@ -1136,21 +1135,8 @@ public class DistributedRegion extends LocalRegion implements
       getLockService(); // create lock service eagerly now
     }
 
-    final IndexUpdater indexUpdater = getIndexUpdater();
-    boolean sqlfGIILockTaken = false;
-    // this try block is to release the SQLF GII lock in finally
-    // which should be done after bucket status will be set
-    // properly in LocalRegion#initialize()
-    try {
      try {
       try {
-        // take the GII lock to avoid missing entries while updating the
-        // index list for SQLFabric (#41330 and others)
-        if (indexUpdater != null) {
-          indexUpdater.lockForGII();
-          sqlfGIILockTaken = true;
-        }
-        
         PersistentMemberID persistentId = null;
         boolean recoverFromDisk = isRecoveryNeeded();
         DiskRegion dskRgn = getDiskRegion();
@@ -1194,11 +1180,6 @@ public class DistributedRegion extends LocalRegion implements
         this.eventTracker.setInitialized();
       }
      }
-    } finally {
-      if (sqlfGIILockTaken) {
-        indexUpdater.unlockForGII();
-      }
-    }
   }
 
   @Override
@@ -2273,13 +2254,6 @@ public class DistributedRegion extends LocalRegion implements
     }
     profile.serialNumber = getSerialNumber();
     profile.regionInitialized = this.isInitialized();
-    if (!this.isUsedForPartitionedRegionBucket()) {
-      profile.memberUnInitialized = getCache().isUnInitializedMember(
-          profile.getDistributedMember());
-    }
-    else {
-      profile.memberUnInitialized = false;
-    }
     profile.persistentID = getPersistentID();
     if(getPersistenceAdvisor() != null) {
       profile.persistenceInitialized = getPersistenceAdvisor().isOnline();
@@ -2485,11 +2459,7 @@ public class DistributedRegion extends LocalRegion implements
     }
     
     if (preferCD) {
-      if (event.hasDelta()) {
-        result = event.getNewValue();
-      } else {
         result = event.getRawNewValueAsHeapObject();
-      }    
     } else {
       result = event.getNewValue();     
     }
@@ -3909,12 +3879,10 @@ public class DistributedRegion extends LocalRegion implements
   /**
    * Used to bootstrap txState.
    * @param key
-   * @return  distributedRegions,
-   * member with parimary bucket for partitionedRegions
+   * @return member with primary bucket for partitionedRegions
    */
   @Override
   public DistributedMember getOwnerForKey(KeyInfo key) {
-    //Asif: fix for  sqlfabric bug 42266
     assert !this.isInternalRegion() || this.isMetaRegionWithTransactions();
     if (!this.getAttributes().getDataPolicy().withStorage()
         || (this.concurrencyChecksEnabled && this.getAttributes()
@@ -4032,8 +4000,7 @@ public class DistributedRegion extends LocalRegion implements
       if (this.randIndex < 0) {
         this.randIndex = PartitionedRegion.rand.nextInt(numProfiles);
       }
-      if (cp.dataPolicy.withReplication() && cp.regionInitialized
-          && !cp.memberUnInitialized) {
+      if (cp.dataPolicy.withReplication() && cp.regionInitialized) {
         if (onlyPersistent && !cp.dataPolicy.withPersistence()) {
           return true;
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegionFunctionStreamingMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegionFunctionStreamingMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegionFunctionStreamingMessage.java
index e51bedd..af54945 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegionFunctionStreamingMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRegionFunctionStreamingMessage.java
@@ -231,8 +231,6 @@ public class DistributedRegionFunctionStreamingMessage extends DistributionMessa
           }
         }
         // Send the reply if the operateOnPartitionedRegion returned true
-        // Fix for hang in dunits on sqlfabric after merge.
-        //ReplyMessage.send(getSender(), this.processorId, rex, dm);
         sendReply(getSender(), this.processorId, dm, rex, null, 0, true, false);
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRemoveAllOperation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRemoveAllOperation.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRemoveAllOperation.java
index eabefdf..0390b29 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRemoveAllOperation.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/DistributedRemoveAllOperation.java
@@ -380,8 +380,7 @@ public class DistributedRemoveAllOperation extends AbstractUpdateOperation // TO
      * {@link PutAllPRMessage#toData(DataOutput)} <br>
      * {@link RemotePutAllMessage#toData(DataOutput)} <br>
      */
-    public final void toData(final DataOutput out, 
-        final boolean requiresRegionContext) throws IOException {
+    public final void toData(final DataOutput out) throws IOException {
       Object key = this.key;
       DataSerializer.writeObject(key, out);
 
@@ -886,11 +885,9 @@ public class DistributedRemoveAllOperation extends AbstractUpdateOperation // TO
      * @param rgn
      *          the region the entry is removed from
      */
-    public void doEntryRemove(RemoveAllEntryData entry, DistributedRegion rgn,
-        boolean requiresRegionContext) {
+    public void doEntryRemove(RemoveAllEntryData entry, DistributedRegion rgn) {
       @Released EntryEventImpl ev = RemoveAllMessage.createEntryEvent(entry, getSender(), 
-          this.context, rgn,
-          requiresRegionContext, this.possibleDuplicate,
+          this.context, rgn, this.possibleDuplicate,
           this.needsRouting, this.callbackArg, true, skipCallbacks);
 //      rgn.getLogWriterI18n().info(LocalizedStrings.DEBUG, "RemoveAllMessage.doEntryRemove sender=" + getSender() +
 //          " event="+ev);
@@ -922,7 +919,6 @@ public class DistributedRemoveAllOperation extends AbstractUpdateOperation // TO
      * @param sender
      * @param context
      * @param rgn
-     * @param requiresRegionContext
      * @param possibleDuplicate
      * @param needsRouting
      * @param callbackArg
@@ -931,13 +927,10 @@ public class DistributedRemoveAllOperation extends AbstractUpdateOperation // TO
     @Retained
     public static EntryEventImpl createEntryEvent(RemoveAllEntryData entry,
         InternalDistributedMember sender, ClientProxyMembershipID context,
-        DistributedRegion rgn, boolean requiresRegionContext, 
+        DistributedRegion rgn,
         boolean possibleDuplicate, boolean needsRouting, Object callbackArg,
         boolean originRemote, boolean skipCallbacks) {
       final Object key = entry.getKey();
-      if (requiresRegionContext) {
-        ((KeyWithRegionContext)key).setRegionContext(rgn);
-      }
       EventID evId = entry.getEventID();
       @Retained EntryEventImpl ev = EntryEventImpl.create(rgn, entry.getOp(),
           key, null/* value */, callbackArg,
@@ -985,13 +978,12 @@ public class DistributedRemoveAllOperation extends AbstractUpdateOperation // TO
       
       rgn.syncBulkOp(new Runnable() {
         public void run() {
-          final boolean requiresRegionContext = rgn.keyRequiresRegionContext();
           for (int i = 0; i < removeAllDataSize; ++i) {
             if (logger.isTraceEnabled()) {
               logger.trace("removeAll processing {} with {}", removeAllData[i], removeAllData[i].versionTag);
             }
             removeAllData[i].setSender(sender);
-            doEntryRemove(removeAllData[i], rgn, requiresRegionContext);
+            doEntryRemove(removeAllData[i], rgn);
           }
         }
       }, ev.getEventId());
@@ -1043,10 +1035,6 @@ public class DistributedRemoveAllOperation extends AbstractUpdateOperation // TO
         EntryVersionsList versionTags = new EntryVersionsList(removeAllDataSize);
 
         boolean hasTags = false;
-        // get the "keyRequiresRegionContext" flag from first element assuming
-        // all key objects to be uniform
-        final boolean requiresRegionContext =
-          (this.removeAllData[0].key instanceof KeyWithRegionContext);
         for (int i = 0; i < this.removeAllDataSize; i++) {
           if (!hasTags && removeAllData[i].versionTag != null) {
             hasTags = true;
@@ -1054,7 +1042,7 @@ public class DistributedRemoveAllOperation extends AbstractUpdateOperation // TO
           VersionTag<?> tag = removeAllData[i].versionTag;
           versionTags.add(tag);
           removeAllData[i].versionTag = null;
-          this.removeAllData[i].toData(out, requiresRegionContext);
+          this.removeAllData[i].toData(out);
           this.removeAllData[i].versionTag = tag;
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryBits.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryBits.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryBits.java
index f95af60..a67a335 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryBits.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryBits.java
@@ -30,16 +30,6 @@ public abstract class EntryBits {
   private static final byte TOMBSTONE = 0x40;
   private static final byte WITH_VERSIONS = (byte)0x80; // oplog entry contains versions 
 
-  /**
-   * Currently for SQLFabric to deserialize byte[][] eagerly in
-   * InitialImageOperation. Can be made a general flag later for all kinds of
-   * objects in CachedDeserializable whose serialization is not expensive but
-   * that are pretty heavy so creating an intermediate byte[] is expensive.
-   * 
-   * This is a transient bit that clashes with on-disk persisted bits.
-   */
-  private static final byte EAGER_DESERIALIZE = 0x20;
-
   public static boolean isSerialized(byte b) {
     return (b & SERIALIZED) != 0;
   }
@@ -79,10 +69,6 @@ public abstract class EntryBits {
     return (b & (INVALID|LOCAL_INVALID|TOMBSTONE)) == 0;
   }
 
-  public static boolean isEagerDeserialize(byte b) {
-    return (b & EntryBits.EAGER_DESERIALIZE) != 0;
-  }
-
   public static byte setSerialized(byte b, boolean isSerialized) {
     return isSerialized ? (byte)(b | SERIALIZED) : (byte)(b & ~SERIALIZED);
   }
@@ -116,12 +102,4 @@ public abstract class EntryBits {
   public static byte getPersistentBits(byte b) {
     return (byte)(b & (SERIALIZED|INVALID|LOCAL_INVALID|TOMBSTONE|WITH_VERSIONS));
   }
-
-  public static byte setEagerDeserialize(byte b) {
-    return (byte)(b | EntryBits.EAGER_DESERIALIZE);
-  }
-
-  public static byte clearEagerDeserialize(byte b) {
-    return (byte)(b & ~EntryBits.EAGER_DESERIALIZE);
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
index 65b2c04..c4849be 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryEventImpl.java
@@ -21,7 +21,6 @@ import com.gemstone.gemfire.*;
 import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.cache.query.IndexMaintenanceException;
 import com.gemstone.gemfire.cache.query.QueryException;
-import com.gemstone.gemfire.cache.query.internal.IndexUpdater;
 import com.gemstone.gemfire.cache.query.internal.index.IndexManager;
 import com.gemstone.gemfire.cache.query.internal.index.IndexProtocol;
 import com.gemstone.gemfire.cache.query.internal.index.IndexUtils;
@@ -33,7 +32,6 @@ import com.gemstone.gemfire.distributed.internal.DistributionMessage;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.*;
 import com.gemstone.gemfire.internal.cache.FilterRoutingInfo.FilterInfo;
-import com.gemstone.gemfire.internal.cache.delta.Delta;
 import com.gemstone.gemfire.internal.cache.lru.Sizeable;
 import com.gemstone.gemfire.internal.cache.partitioned.PartitionMessage;
 import com.gemstone.gemfire.internal.cache.partitioned.PutMessage;
@@ -92,7 +90,6 @@ public class EntryEventImpl
   private byte[] cachedSerializedNewValue = null;
   @Retained(ENTRY_EVENT_OLD_VALUE)
   private Object oldValue = null;
-  protected Delta delta = null;
  
   protected short eventFlags = 0x0000;
 
@@ -141,13 +138,6 @@ public class EntryEventImpl
   protected ClientProxyMembershipID context = null;
   
   /**
-   * A custom context object that can be used for any other contextual
-   * information. Currently used by SQL Fabric to pass around evaluated rows
-   * from raw byte arrays and routing object.
-   */
-  private transient Object contextObj = null;
-
-  /**
    * this holds the bytes representing the change in value effected by this
    * event.  It is used when the value implements the Delta interface.
    */
@@ -165,8 +155,6 @@ public class EntryEventImpl
   /** version tag for concurrency checks */
   protected VersionTag versionTag;
 
-  private transient boolean isPutDML = false;
-
   /** boolean to indicate that the RegionEntry for this event has been evicted*/
   private transient boolean isEvicted = false;
   
@@ -191,7 +179,7 @@ public class EntryEventImpl
     this.txId = (TXId)DataSerializer.readObject(in);
 
     if (in.readBoolean()) {     // isDelta
-      this.delta = (Delta)DataSerializer.readObject(in);
+      assert false : "isDelta should never be true";
     }
     else {
       // OFFHEAP Currently values are never deserialized to off heap memory. If that changes then this code needs to change.
@@ -247,10 +235,7 @@ public class EntryEventImpl
     this.op = op;
     this.keyInfo = this.region.getKeyInfo(key, newVal, callbackArgument);
 
-    if (newVal instanceof Delta) {
-      this.delta = (Delta)newVal;
-    }
-    else if (!Token.isInvalid(newVal)) {
+    if (!Token.isInvalid(newVal)) {
       basicSetNewValue(newVal);
     }
 
@@ -299,7 +284,6 @@ public class EntryEventImpl
     this.newValueBytes = other.newValueBytes;
     this.cachedSerializedNewValue = other.cachedSerializedNewValue;
     this.re = other.re;
-    this.delta = other.delta;
     if (setOldValue) {
       retainAndSetOldValue(other.basicGetOldValue());
       this.oldValueBytes = other.oldValueBytes;
@@ -781,25 +765,19 @@ public class EntryEventImpl
 
   /**
    * Like getRawNewValue except that if the result is an off-heap reference then copy it to the heap.
-   * ALERT: If there is a Delta, returns that, not the (applied) new value.
    * Note: to prevent the heap copy use getRawNewValue instead
    */
   public final Object getRawNewValueAsHeapObject() {
-    if (this.delta != null) {
-      return this.delta;
-    }
     return OffHeapHelper.getHeapForm(OffHeapHelper.copyIfNeeded(basicGetNewValue()));
   }
   
   /**
-   * If new value is a Delta return it.
-   * Else if new value is off-heap return the StoredObject form (unretained OFF_HEAP_REFERENCE). 
+   * If new value is off-heap return the StoredObject form (unretained OFF_HEAP_REFERENCE). 
    * Its refcount is not inced by this call and the returned object can only be safely used for the lifetime of the EntryEventImpl instance that returned the value.
    * Else return the raw form.
    */
   @Unretained(ENTRY_EVENT_NEW_VALUE)
   public final Object getRawNewValue() {
-    if (this.delta != null) return this.delta;
     return basicGetNewValue();
   }
 
@@ -808,39 +786,6 @@ public class EntryEventImpl
     return basicGetNewValue();
   }
   
-  /**
-   * Returns the delta that represents the new value; null if no delta.
-   * @return the delta that represents the new value; null if no delta.
-   */
-  public final Delta getDeltaNewValue() {
-    return this.delta;
-  }
-
-  /**
-   *  Applies the delta 
-   */
-  private Object applyDeltaWithCopyOnRead(boolean doCopyOnRead) {
-    //try {
-      if (applyDelta(true)) {
-        Object applied = basicGetNewValue();
-        // if applyDelta returns true then newValue should not be off-heap
-        assert !(applied instanceof StoredObject);
-        if (applied == this.oldValue && doCopyOnRead) {
-          applied = CopyHelper.copy(applied);
-        }
-        return applied;
-      }
-    //} catch (EntryNotFoundException ex) {
-      // only (broken) product code has the opportunity to call this before
-      // this.oldValue is set. If oldValue is not set yet, then
-      // we most likely haven't synchronized on the region entry yet.
-      // (If we have, then make sure oldValue is set before
-      // calling this method).
-      //throw new AssertionError("too early to call getNewValue");
-    //}
-    return null;
-  }
-
   @Released(ENTRY_EVENT_NEW_VALUE)
   protected void basicSetNewValue(@Retained(ENTRY_EVENT_NEW_VALUE) Object v) {
     if (v == this.newValue) return;
@@ -1003,23 +948,6 @@ public class EntryEventImpl
   public final Object getNewValue() {
     
     boolean doCopyOnRead = getRegion().isCopyOnRead();
-    try {
-      if (applyDelta(true)) {
-        @Unretained(ENTRY_EVENT_NEW_VALUE)
-        Object applied = basicGetNewValue();
-        if (applied == this.oldValue && doCopyOnRead) {
-          applied = CopyHelper.copy(applied);
-        }
-        return applied;
-      }
-    } catch (EntryNotFoundException ex) {
-      // only (broken) product code has the opportunity to call this before
-      // this.oldValue is set. If oldValue is not set yet, then
-      // we most likely haven't synchronized on the region entry yet.
-      // (If we have, then make sure oldValue is set before
-      // calling this method).
-      throw new AssertionError("too early to call getNewValue");
-    }
     Object nv = basicGetNewValue();
     if (nv != null) {
       if (nv == Token.NOT_AVAILABLE) {
@@ -1055,44 +983,9 @@ public class EntryEventImpl
     return StringUtils.forceToString(basicGetOldValue());
   }
   
-  protected boolean applyDelta(boolean throwOnNullOldValue)
-      throws EntryNotFoundException {
-    if (this.newValue != null || this.delta == null) {
-      return false;
-    }
-    if (this.oldValue == null) {
-      if (throwOnNullOldValue) {
-        // !!!:ezoerner:20080611 It would be nice if the client got this
-        // exception
-        throw new EntryNotFoundException(
-            "Cannot apply a delta without an existing value");
-      }
-      return false;
-    }
-    // swizzle BucketRegion in event for Delta.
-    // !!!:ezoerner:20090602 this is way ugly; this whole class severely
-    // needs refactoring
-    LocalRegion originalRegion = this.region;
-    try {
-      if (originalRegion instanceof BucketRegion) {
-        this.region = ((BucketRegion)this.region).getPartitionedRegion();
-      }
-      basicSetNewValue(this.delta.apply(this));
-    } finally {
-      this.region = originalRegion;
-    }
-    return true;
-  }
-
   /** Set a deserialized value */
   public final void setNewValue(@Retained(ENTRY_EVENT_NEW_VALUE) Object obj) {
-    if (obj instanceof Delta) {
-      this.delta = (Delta)obj;
-      basicSetNewValue(null);
-    }
-    else {
-      basicSetNewValue(obj);
-    }
+    basicSetNewValue(obj);
   }
 
   public TransactionId getTransactionId()
@@ -1384,34 +1277,11 @@ public class EntryEventImpl
   }
 
   /**
-   * If applyDelta is true then first attempt to apply a delta (if we have one) and return the value.
-   * Else if new value is a Delta return it.
-   * Else if new value is off-heap return the StoredObject form (unretained OFF_HEAP_REFERENCE). 
-   * Its refcount is not inced by this call and the returned object can only be safely used for the lifetime of the EntryEventImpl instance that returned the value.
-   * Else return the raw form.
-   */
-  @Unretained(ENTRY_EVENT_NEW_VALUE)
-  public final Object getRawNewValue(boolean applyDelta) {
-    if (applyDelta) {
-      boolean doCopyOnRead = getRegion().isCopyOnRead();
-      Object newValueWithDelta = applyDeltaWithCopyOnRead(doCopyOnRead);
-      if (newValueWithDelta != null) {
-        return newValueWithDelta;
-      }
-      // if applyDelta is true and we have already applied the delta then
-      // just return the applied value instead of the delta object.
-      @Unretained(ENTRY_EVENT_NEW_VALUE)
-      Object newValue = basicGetNewValue();
-      if (newValue != null) return newValue;
-    }
-    return getRawNewValue();
-  }
-  /**
    * Just like getRawNewValue(true) except if the raw new value is off-heap deserialize it.
    */
   @Unretained(ENTRY_EVENT_NEW_VALUE)
   public final Object getNewValueAsOffHeapDeserializedOrRaw() {
-    Object result = getRawNewValue(true);
+    Object result = getRawNewValue();
     if (result instanceof StoredObject) {
       result = ((StoredObject) result).getDeserializedForReading();
     }
@@ -1448,7 +1318,6 @@ public class EntryEventImpl
   }
   
   public final Object getDeserializedValue() {
-    if (this.delta == null) {
       final Object val = basicGetNewValue();
       if (val instanceof CachedDeserializable) {
         return ((CachedDeserializable)val).getDeserializedForReading();
@@ -1456,16 +1325,11 @@ public class EntryEventImpl
       else {
         return val;
       }
-    }
-    else {
-      return this.delta;
-    }
   }
 
   public final byte[] getSerializedValue() {
     if (this.newValueBytes == null) {
       final Object val;
-      if (this.delta == null) {
         val = basicGetNewValue();
         if (val instanceof byte[]) {
           return (byte[])val;
@@ -1473,10 +1337,6 @@ public class EntryEventImpl
         else if (val instanceof CachedDeserializable) {
           return ((CachedDeserializable)val).getSerializedValue();
         }
-      }
-      else {
-        val = this.delta;
-      }
       try {
         return CacheServerHelper.serialize(val);
       } catch (IOException ioe) {
@@ -1508,11 +1368,6 @@ public class EntryEventImpl
     if (isSynced) {
       this.setSerializationDeferred(false);
     }
-    else if (obj == null && this.delta != null) {
-      // defer serialization until setNewValueInRegion
-      this.setSerializationDeferred(true);
-      return;
-    }
     basicSetNewValue(getCachedDeserializable(obj, this));
   }
 
@@ -1527,12 +1382,11 @@ public class EntryEventImpl
                             || obj == Token.NOT_AVAILABLE
                             || Token.isInvalidOrRemoved(obj)
                             // don't serialize delta object already serialized
-                            || obj instanceof com.gemstone.gemfire.Delta
-                            || obj instanceof Delta) { // internal delta
+                            || obj instanceof com.gemstone.gemfire.Delta) { // internal delta
       return obj;
     }
     final CachedDeserializable cd;
-    // avoid unneeded serialization of byte[][] used by SQLFabric that
+    // avoid unneeded serialization of byte[][] that
     // will end up being deserialized in any case (serialization is cheap
     //   for byte[][] anyways)
     if (obj instanceof byte[][]) {
@@ -1567,18 +1421,7 @@ public class EntryEventImpl
   public final void setSerializedNewValue(byte[] serializedValue) {
     Object newVal = null;
     if (serializedValue != null) {
-      if (CachedDeserializableFactory.preferObject()) {
-        newVal = deserialize(serializedValue);
-      } else {
-        newVal = CachedDeserializableFactory.create(serializedValue);
-      }
-      if (newVal instanceof Delta) {
-        this.delta = (Delta)newVal;
-        newVal = null;
-        // We need the newValueBytes field and the newValue field to be in sync.
-        // In the case of non-null delta set both fields to null.
-        serializedValue = null;
-      }
+      newVal = CachedDeserializableFactory.create(serializedValue);
     }
     this.newValueBytes = serializedValue;
     basicSetNewValue(newVal);
@@ -1588,10 +1431,7 @@ public class EntryEventImpl
   public void setSerializedOldValue(byte[] serializedOldValue){
     this.oldValueBytes = serializedOldValue;
     final Object ov;
-    if (CachedDeserializableFactory.preferObject()) {
-      ov = deserialize(serializedOldValue);
-    }
-    else if (serializedOldValue != null) {
+    if (serializedOldValue != null) {
       ov = CachedDeserializableFactory.create(serializedOldValue);
     }
     else {
@@ -1705,12 +1545,6 @@ public class EntryEventImpl
     
     // put in newValue
 
-    if (applyDelta(this.op.isCreate())) {
-      if (this.isSerializationDeferred()) {
-        makeSerializedNewValue(true);
-      }
-    }
-
     // If event contains new value, then it may mean that the delta bytes should
     // not be applied. This is possible if the event originated locally.
     if (this.deltaBytes != null && this.newValue == null) {
@@ -1749,7 +1583,7 @@ public class EntryEventImpl
       basicSetNewValue(v);
     }
 
-    Object preparedV = reentry.prepareValueForCache(this.region, v, this, this.hasDelta());
+    Object preparedV = reentry.prepareValueForCache(this.region, v, this, false);
     if (preparedV != v) {
       v = preparedV;
       if (v instanceof StoredObject) {
@@ -1790,29 +1624,9 @@ public class EntryEventImpl
         }
       }
     }
-    final IndexUpdater indexUpdater = this.region.getIndexUpdater();
-    if (indexUpdater != null) {
-      final LocalRegion indexRegion;
-      if (owner != null) {
-        indexRegion = owner;
-      }
-      else {
-        indexRegion = this.region;
-      }
-      try {
-        indexUpdater.onEvent(indexRegion, this, reentry);
-        calledSetValue = true;
-        reentry.setValueWithTombstoneCheck(v, this); // already called prepareValueForCache
-        success = true;
-      } finally {
-        indexUpdater.postEvent(indexRegion, this, reentry, success);
-      }
-    }
-    else {
-      calledSetValue = true;
-      reentry.setValueWithTombstoneCheck(v, this); // already called prepareValueForCache
-      success = true;
-    }
+    calledSetValue = true;
+    reentry.setValueWithTombstoneCheck(v, this); // already called prepareValueForCache
+    success = true;
     } finally {
       if (!success && reentry instanceof OffHeapRegionEntry && v instanceof StoredObject) {
         OffHeapRegionEntryHelper.releaseEntry((OffHeapRegionEntry)reentry, (StoredObject)v);
@@ -1950,8 +1764,6 @@ public class EntryEventImpl
         // there must be a nearSidePendingValue
         processDeltaBytes(tx.getNearSidePendingValue());
         v = basicGetNewValue();
-      } else if (this.delta != null) {
-        v = this.delta;
       } else {
         v = isLocalInvalid() ? Token.LOCAL_INVALID : Token.INVALID;
       }
@@ -2053,12 +1865,6 @@ public class EntryEventImpl
   /** Return true if new value available */
   public boolean hasNewValue() {
     Object tmp = this.newValue;
-    if (tmp == null && hasDelta()) {
-      // ???:ezoerner:20080611 what if applying the delta would produce
-      // null or (strangely) NOT_AVAILABLE.. do we need to apply it here to
-      // find out?
-      return true;
-    }
     return  tmp != null && tmp != Token.NOT_AVAILABLE;
   }
 
@@ -2069,16 +1875,6 @@ public class EntryEventImpl
     return this.oldValue instanceof Token;
   }
 
-  /**
-   * This should only be used in case of internal delta and <B>not for Delta of
-   * Delta Propagation feature</B>.
-   * 
-   * @return boolean
-   */
-  public boolean hasDelta() {
-    return (this.delta != null);
-  }
-
   public boolean isOldValueAvailable() {
     if (isOriginRemote() && this.region.isProxy()) {
       return false;
@@ -2297,12 +2093,8 @@ public class EntryEventImpl
     DataSerializer.writeObject(this.txId, out);
 
     {
-      boolean isDelta = this.delta != null;
-      out.writeBoolean(isDelta);
-      if (isDelta) {
-        DataSerializer.writeObject(this.delta, out);
-      }
-      else {
+      out.writeBoolean(false);
+      {
         Object nv = basicGetNewValue();
         boolean newValueSerialized = nv instanceof CachedDeserializable;
         if (newValueSerialized) {
@@ -2422,7 +2214,6 @@ public class EntryEventImpl
   public int getNewValSizeForPR()
   {
     int newSize = 0;
-    applyDelta(false);
     Object v = basicGetNewValue();
     if (v != null) {
       try {
@@ -2664,14 +2455,6 @@ public class EntryEventImpl
     return this;
   }
 
-  public final void setContextObject(Object ctx) {
-    this.contextObj = ctx;
-  }
-
-  public final Object getContextObject() {
-    return this.contextObj;
-  }
-
   /**
    * @return the keyInfo
    */
@@ -2994,12 +2777,4 @@ public class EntryEventImpl
   public boolean isOldValueOffHeap() {
     return isOffHeapReference(this.oldValue);
   }
-
-  public final boolean isPutDML() {
-    return this.isPutDML;
-  }
-
-  public final void setPutDML(boolean val) {
-    this.isPutDML = val;
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryOperationImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryOperationImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryOperationImpl.java
index 16215ac..4b757fb 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryOperationImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/EntryOperationImpl.java
@@ -38,7 +38,7 @@ public class EntryOperationImpl implements EntryOperation {
 
   private final Object key;
 
-  private Object callbackArgument = Token.NOT_AVAILABLE;
+  private final Object callbackArgument;
 
   public EntryOperationImpl(Region region, Operation operation, Object key,
       Object value, Object callbackArgument) {
@@ -102,17 +102,4 @@ public class EntryOperationImpl implements EntryOperation {
   public Object getRawNewValue() {
     return this.value;
   }
-
-  /**
-   * Method for internal use. (Used by SQLFabric)
-   */
-  public void setCallbackArgument(Object newCallbackArgument) {
-    if (this.callbackArgument instanceof WrappedCallbackArgument) {
-      ((WrappedCallbackArgument)this.callbackArgument)
-          .setOriginalCallbackArgument(newCallbackArgument);
-    }
-    else {
-      this.callbackArgument = newCallbackArgument;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
index 5355a2b..186ebbc 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
@@ -17,7 +17,69 @@
 
 package com.gemstone.gemfire.internal.cache;
 
-import com.gemstone.gemfire.*;
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.PrintStream;
+import java.io.Reader;
+import java.io.StringBufferInputStream;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.ServiceLoader;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import javax.naming.Context;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.CancelCriterion;
+import com.gemstone.gemfire.CancelException;
+import com.gemstone.gemfire.ForcedDisconnectException;
+import com.gemstone.gemfire.GemFireCacheException;
+import com.gemstone.gemfire.InternalGemFireError;
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.admin.internal.SystemMemberCacheEventProcessor;
 import com.gemstone.gemfire.cache.*;
 import com.gemstone.gemfire.cache.TimeoutException;
@@ -107,19 +169,6 @@ import com.gemstone.gemfire.pdx.internal.TypeRegistry;
 import com.gemstone.gemfire.redis.GemFireRedisServer;
 import com.sun.jna.Native;
 import com.sun.jna.Platform;
-import org.apache.logging.log4j.Logger;
-
-import javax.naming.Context;
-import java.io.*;
-import java.net.InetSocketAddress;
-import java.net.URL;
-import java.net.UnknownHostException;
-import java.util.*;
-import java.util.Map.Entry;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
 
 // @todo somebody Come up with more reasonable values for {@link #DEFAULT_LOCK_TIMEOUT}, etc.
 /**
@@ -414,18 +463,6 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
 
   private volatile boolean isShutDownAll = false;
 
-  /**
-   * Set of members that are not yet ready. Currently used by SQLFabric during initial DDL replay to indicate that the
-   * member should not be chosen for primary buckets.
-   */
-  private final HashSet<InternalDistributedMember> unInitializedMembers = new HashSet<InternalDistributedMember>();
-
-  /**
-   * Set of {@link BucketAdvisor}s for this node that are pending for volunteer for primary due to uninitialized node
-   * (SQLFabric DDL replay in progress).
-   */
-  private final LinkedHashSet<BucketAdvisor> deferredVolunteerForPrimary = new LinkedHashSet<BucketAdvisor>();
-
   private final ResourceAdvisor resourceAdvisor;
   private final JmxManagerAdvisor jmxAdvisor;
 
@@ -4911,76 +4948,6 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     return this.regionsInDestroy.get(path);
   }
 
-  /**
-   * Mark a node as initialized or not initialized. Used by SQLFabric to avoid creation of buckets or routing of
-   * operations/functions on a node that is still in the DDL replay phase.
-   */
-  public boolean updateNodeStatus(InternalDistributedMember member, boolean initialized) {
-    HashSet<BucketAdvisor> advisors = null;
-    synchronized (this.unInitializedMembers) {
-      if (initialized) {
-        if (this.unInitializedMembers.remove(member)) {
-          if (member.equals(getMyId())) {
-            // don't invoke volunteerForPrimary() inside the lock since
-            // BucketAdvisor will also require the lock after locking itself
-            advisors = new HashSet<BucketAdvisor>(this.deferredVolunteerForPrimary);
-            this.deferredVolunteerForPrimary.clear();
-          }
-        } else {
-          return false;
-        }
-      } else {
-        return this.unInitializedMembers.add(member);
-      }
-    }
-    if (advisors != null) {
-      for (BucketAdvisor advisor : advisors) {
-        if (logger.isDebugEnabled()) {
-          logger.debug("Invoking volunteer for primary for deferred bucket " + "post SQLFabric DDL replay for BucketAdvisor: {}",  advisor);
-        }
-        advisor.volunteerForPrimary();
-      }
-    }
-    return true;
-  }
-
-  /**
-   * Return true if this node is still not initialized else false.
-   */
-  public boolean isUnInitializedMember(InternalDistributedMember member) {
-    synchronized (this.unInitializedMembers) {
-      return this.unInitializedMembers.contains(member);
-    }
-  }
-
-  /**
-   * Return false for volunteer primary if this node is not currently initialized. Also adds the {@link BucketAdvisor}
-   * to a list that will be replayed once this node is initialized.
-   */
-  public boolean doVolunteerForPrimary(BucketAdvisor advisor) {
-    synchronized (this.unInitializedMembers) {
-      if (!this.unInitializedMembers.contains(getMyId())) {
-        return true;
-      }
-      if (logger.isDebugEnabled()) {
-        logger.debug("Deferring volunteer for primary due to uninitialized " + "node (SQLFabric DDL replay) for BucketAdvisor: {}", advisor);
-      }
-      this.deferredVolunteerForPrimary.add(advisor);
-      return false;
-    }
-  }
-
-  /**
-   * Remove all the uninitialized members from the given collection.
-   */
-  public final void removeUnInitializedMembers(Collection<InternalDistributedMember> members) {
-    synchronized (this.unInitializedMembers) {
-      for (final InternalDistributedMember m : this.unInitializedMembers) {
-        members.remove(m);
-      }
-    }
-  }
-
   public TombstoneService getTombstoneService() {
     return this.tombstoneService;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GridAdvisor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GridAdvisor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GridAdvisor.java
index ec5fc4e..ff7dea1 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GridAdvisor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GridAdvisor.java
@@ -249,7 +249,7 @@ public abstract class GridAdvisor extends DistributionAdvisor {
     private String host;
 
     /**
-     * SQLFabric uses a negative port value when creating a fake profile meant
+     * a negative port value is used when creating a fake profile meant
      * to only gather information about all available locators.
      */
     private int port;
@@ -308,7 +308,7 @@ public abstract class GridAdvisor extends DistributionAdvisor {
         if(advisee != null && advisee.getProfile().equals(this)) {
           continue;
         }
-        // negative value for port used by SQLFabric to indicate fake profile
+        // negative value for port indicates fake profile
         // meant to only gather remote profiles during profile exchange
         if (this.port > 0) {
           handleDistributionAdvisee(advisee, removeProfile, exchangeProfiles,
@@ -337,7 +337,7 @@ public abstract class GridAdvisor extends DistributionAdvisor {
             if(bsi.getProfile().equals(this)) {
               continue;
             }
-            // negative value for port used by SQLFabric to indicate fake
+            // negative value for port indicates fake
             // profile meant to only gather remote profiles during profile
             // exchange
             if (this.port > 0) {



[3/5] incubator-geode git commit: GEODE-1464: remove sqlf code

Posted by ds...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageOperation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageOperation.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageOperation.java
index f0836d4..55bdde4 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageOperation.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InitialImageOperation.java
@@ -23,7 +23,6 @@ import com.gemstone.gemfire.cache.Operation;
 import com.gemstone.gemfire.cache.RegionDestroyedException;
 import com.gemstone.gemfire.cache.query.internal.CqStateImpl;
 import com.gemstone.gemfire.cache.query.internal.DefaultQueryService;
-import com.gemstone.gemfire.cache.query.internal.IndexUpdater;
 import com.gemstone.gemfire.cache.query.internal.cq.CqService;
 import com.gemstone.gemfire.cache.query.internal.cq.ServerCQ;
 import com.gemstone.gemfire.distributed.DistributedMember;
@@ -764,10 +763,6 @@ public class InitialImageOperation  {
       if (entryCount <= 1000 && isDebugEnabled) {
         keys = new HashSet();
       }
-      final boolean keyRequiresRegionContext = this.region
-          .keyRequiresRegionContext();
-      // get SQLF index manager for the case of recovery from disk
-      final IndexUpdater indexUpdater = this.region.getIndexUpdater();
       final ByteArrayDataInput in = new ByteArrayDataInput();
       for (int i = 0; i < entryCount; i++) {
         // stream is null-terminated
@@ -816,33 +811,7 @@ public class InitialImageOperation  {
         Object tmpValue = entry.value;
         byte[] tmpBytes = null;
 
-        if (keyRequiresRegionContext) {
-          final KeyWithRegionContext key = (KeyWithRegionContext)entry.key;
-          Object keyObject = tmpValue;
-          if (tmpValue != null) {
-            if (entry.isEagerDeserialize()) {
-              tmpValue = CachedDeserializableFactory.create(tmpValue,
-                  CachedDeserializableFactory.getArrayOfBytesSize(
-                      (byte[][])tmpValue, true));
-              entry.setSerialized(false);
-            }
-            else if (entry.isSerialized()) {
-              tmpBytes = (byte[])tmpValue;
-              // force deserialization for passing to key
-              keyObject = EntryEventImpl.deserialize(tmpBytes,
-                  remoteVersion, in);
-              tmpValue = CachedDeserializableFactory.create(keyObject,
-                  CachedDeserializableFactory.getByteSize(tmpBytes));
-              entry.setSerialized(false);
-            }
-            else {
-              tmpBytes = (byte[])tmpValue;
-            }
-          }
-          key.setRegionContext(this.region);
-          entry.key = key.afterDeserializationWithValue(keyObject);
-        }
-        else {
+        {
           if (tmpValue instanceof byte[]) {
             tmpBytes = (byte[])tmpValue;
           }
@@ -879,32 +848,6 @@ public class InitialImageOperation  {
                 //actually are equal, keep don't put the received
                 //entry into the cache (this avoids writing a record to disk)
                 if(entriesEqual) {
-                  // explicit SQLF index maintenance here since
-                  // it was not done during recovery from disk
-                  if (indexUpdater != null && !Token.isInvalidOrRemoved(tmpValue)) {
-                    boolean success = false;
-                    if (entry.isSerialized()) {
-                      tmpValue = CachedDeserializableFactory
-                          .create((byte[])tmpValue);
-                    }
-                    // dummy EntryEvent to pass for SQLF index maintenance
-                    @Released final EntryEventImpl ev = EntryEventImpl.create(this.region,
-                        Operation.CREATE, null, null, null, true, null, false, false);
-                    try {
-                    ev.setKeyInfo(this.region.getKeyInfo(entry.key,
-                        tmpValue, null));
-                    ev.setNewValue(tmpValue);
-                    try {
-                      indexUpdater.onEvent(this.region, ev, re);
-                      success = true;
-                    } finally {
-                      indexUpdater.postEvent(this.region, ev, re,
-                          success);
-                    }
-                    } finally {
-                      ev.release();
-                    }
-                  }
                   continue;
                 }
                 if (entry.isSerialized() && !Token.isInvalidOrRemoved(tmpValue)) {
@@ -1864,7 +1807,6 @@ public class InitialImageOperation  {
 
       List chunkEntries = null;
       chunkEntries = new InitialImageVersionedEntryList(rgn.concurrencyChecksEnabled, MAX_ENTRIES_PER_CHUNK);
-      final boolean keyRequiresRegionContext = rgn.keyRequiresRegionContext();
       DiskRegion dr = rgn.getDiskRegion();
       if( dr!=null ){
         dr.setClearCountReference();
@@ -1926,9 +1868,6 @@ public class InitialImageOperation  {
                     entry = new InitialImageOperation.Entry();
                     entry.key = key;
                     entry.setVersionTag(stamp.asVersionTag());
-                    if (keyRequiresRegionContext) {
-                      entry.setEagerDeserialize();
-                    }
                     fillRes = mapEntry.fillInValue(rgn, entry, in, rgn.getDistributionManager());
                     if (versionVector != null) {
                       if (logger.isTraceEnabled(LogMarker.GII)) {
@@ -1939,9 +1878,6 @@ public class InitialImageOperation  {
                 } else {
                   entry = new InitialImageOperation.Entry();
                   entry.key = key;
-                  if (keyRequiresRegionContext) {
-                    entry.setEagerDeserialize();
-                  }
                   fillRes = mapEntry.fillInValue(rgn, entry, in, rgn.getDistributionManager());
                 }
               }
@@ -1961,11 +1897,6 @@ public class InitialImageOperation  {
               entry.setLastModified(rgn.getDistributionManager(), mapEntry
                   .getLastModified());
             }
-            if (keyRequiresRegionContext) {
-              entry.key = ((KeyWithRegionContext)key)
-                  .beforeSerializationWithValue(entry.isInvalid()
-                      || entry.isLocalInvalid());
-            }
 
             chunkEntries.add(entry);
             currentChunkSize += entry.calcSerializedSize();
@@ -2952,18 +2883,6 @@ public class InitialImageOperation  {
       this.entryBits = EntryBits.setSerialized(this.entryBits, isSerialized);
     }
 
-    public boolean isEagerDeserialize() {
-      return EntryBits.isEagerDeserialize(this.entryBits);
-    }
-
-    void setEagerDeserialize() {
-      this.entryBits = EntryBits.setEagerDeserialize(this.entryBits);
-    }
-
-    void clearEagerDeserialize() {
-      this.entryBits = EntryBits.clearEagerDeserialize(this.entryBits);
-    }
-
     public boolean isInvalid() {
       return (this.value == null) && !EntryBits.isLocalInvalid(this.entryBits);
     }
@@ -3005,12 +2924,7 @@ public class InitialImageOperation  {
       out.writeByte(flags);
       DataSerializer.writeObject(this.key, out);
       if (!EntryBits.isTombstone(this.entryBits)) {
-        if (!isEagerDeserialize()) {
-          DataSerializer.writeObjectAsByteArray(this.value, out);
-        }
-        else {
-          DataSerializer.writeArrayOfByteArrays((byte[][])this.value, out);
-        }
+        DataSerializer.writeObjectAsByteArray(this.value, out);
       }
       out.writeLong(this.lastModified);
       if (this.versionTag != null) {
@@ -3030,11 +2944,7 @@ public class InitialImageOperation  {
       if (EntryBits.isTombstone(this.entryBits)) {
         this.value = Token.TOMBSTONE;
       } else {
-        if (!isEagerDeserialize()) {
-          this.value = DataSerializer.readByteArray(in);
-        } else {
-          this.value = DataSerializer.readArrayOfByteArrays(in);
-        }
+        this.value = DataSerializer.readByteArray(in);
       }
       this.lastModified = in.readLong();
       if ((flags & HAS_VERSION) != 0) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InternalRegionArguments.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InternalRegionArguments.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InternalRegionArguments.java
index 3a254d5..c403231 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InternalRegionArguments.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InternalRegionArguments.java
@@ -22,7 +22,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import com.gemstone.gemfire.cache.query.internal.IndexUpdater;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.cache.LocalRegion.TestCallable;
 import com.gemstone.gemfire.internal.cache.partitioned.RegionAdvisor;
@@ -58,8 +57,6 @@ public final class InternalRegionArguments
   private DiskRegion diskRegion;
   private PartitionedRegion partitionedRegion;
   private TestCallable testCallable;
-  private IndexUpdater indexUpdater;
-  private boolean keyRequiresRegionContext;
 
   private AbstractGatewaySender parallelGatewaySender;
   private AbstractGatewaySender serialGatewaySender;
@@ -233,25 +230,6 @@ public final class InternalRegionArguments
     return this.testCallable;
   }
 
-  // SQLFabric index manager
-  public IndexUpdater getIndexUpdater() {
-    return this.indexUpdater;
-  }
-
-  public InternalRegionArguments setIndexUpdater(IndexUpdater indexUpdater) {
-    this.indexUpdater = indexUpdater;
-    return this;
-  }
-
-  public boolean keyRequiresRegionContext() {
-    return this.keyRequiresRegionContext;
-  }
-
-  public InternalRegionArguments setKeyRequiresRegionContext(boolean v) {
-    this.keyRequiresRegionContext = v;
-    return this;
-  }
-
   public InternalRegionArguments setUserAttribute(Object userAttr) {
     this.userAttribute = userAttr;
     return this;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InvalidateOperation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InvalidateOperation.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InvalidateOperation.java
index 6e1d91e..1742ad3 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InvalidateOperation.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InvalidateOperation.java
@@ -112,9 +112,6 @@ public class InvalidateOperation extends DistributedCacheOperation
     @Retained
     protected InternalCacheEvent createEvent(DistributedRegion rgn)
         throws EntryNotFoundException {
-      if (rgn.keyRequiresRegionContext()) {
-        ((KeyWithRegionContext)this.key).setRegionContext(rgn);
-      }
       @Retained EntryEventImpl ev = EntryEventImpl.create(
          rgn, getOperation(), this.key,
          null, this.callbackArg, true, getSender());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/KeyInfo.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/KeyInfo.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/KeyInfo.java
index 30f30fc..3065b47 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/KeyInfo.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/KeyInfo.java
@@ -40,9 +40,8 @@ public class KeyInfo {
   private Object callbackArg;
   private int bucketId;
 
-  // Rahul: The value field is add since Sqlf Partition resolver also relies on the value
-  // part to calculate the routing object if the table is not partitioned on 
-  // primary key.
+  // The value field is added since a Partition resolver could also rely on the value
+  // part to calculate the routing object
   @Retained(ENTRY_EVENT_NEW_VALUE)
   private final Object value;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/KeyWithRegionContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/KeyWithRegionContext.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/KeyWithRegionContext.java
deleted file mode 100644
index b28a551..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/KeyWithRegionContext.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package com.gemstone.gemfire.internal.cache;
-
-import com.gemstone.gemfire.internal.DataSerializableFixedID;
-
-/**
- * Interface that can be implemented by region keys to allow passing the region
- * after deserialization for any region specific initialization. Note that the
- * {@link LocalRegion#setKeyRequiresRegionContext(boolean)} should also be set
- * for {@link #setRegionContext(LocalRegion)} to be invoked by the GemFire
- * layer. It is required that either all keys of the region implement this
- * interface (and the flag
- * {@link LocalRegion#setKeyRequiresRegionContext(boolean)} is set) or none do.
- * 
- * Currently used by SQLFabric for the optimized
- * <code>CompactCompositeRegionKey</code> key implementations that keeps the key
- * as a reference to the raw row bytes and requires a handle of the table schema
- * to interpret those in hashCode/equals/compareTo methods that have no region
- * context information.
- * 
- */
-public interface KeyWithRegionContext extends DataSerializableFixedID {
-
-  /**
-   * Pass the region of the key for setting up of any region specific context
-   * for the key. In case of recovery from disk the region may not have been
-   * fully initialized yet, so the implementation needs to take that into
-   * consideration.
-   * 
-   * @param region
-   *          the region of this key
-   */
-  public void setRegionContext(LocalRegion region);
-
-  /**
-   * Changes required to be done to the key, if any, to optimize serialization
-   * for sending across when value is also available.
-   * 
-   * SQLFabric will make the value bytes as null in the key so as to avoid
-   * serializing the row twice.
-   */
-  public KeyWithRegionContext beforeSerializationWithValue(boolean valueIsToken);
-
-  /**
-   * Changes required to be done to the key, if any, to after deserializing the
-   * key in reply with value available. The value is required to be provided in
-   * deserialized format (e.g. for {@link CachedDeserializable}s the
-   * deserialized value being wrapped must be passed).
-   * 
-   * SQLFabric will restore the value bytes that were set as null in
-   * {@link #beforeSerializationWithValue}.
-   */
-  public KeyWithRegionContext afterDeserializationWithValue(Object val);
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ListOfDeltas.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ListOfDeltas.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ListOfDeltas.java
deleted file mode 100644
index 6592863..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ListOfDeltas.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.gemstone.gemfire.internal.cache;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import com.gemstone.gemfire.InternalGemFireException;
-import com.gemstone.gemfire.cache.EntryEvent;
-import com.gemstone.gemfire.internal.cache.delta.Delta;
-
-
-/**
- * 
- *
- */
-public final class ListOfDeltas implements Delta {
-
-  private  List<Delta> listOfDeltas;
-  transient private int deltaAppliedIndex = 0;
-  public ListOfDeltas(final int size) {
-    this.listOfDeltas = new ArrayList<Delta>(size);
-  }
-
-  public ListOfDeltas(Delta deltaObj) {
-    this.listOfDeltas = new ArrayList<Delta>();
-    this.listOfDeltas.add(deltaObj);
-  }
-  
-  public ListOfDeltas() {    
-  }
-
-  
-
-  public Object apply(EntryEvent ev) {
-    if (ev != null && ev instanceof EntryEventImpl) {
-      EntryEventImpl putEvent = (EntryEventImpl)ev;
-      int last = this.listOfDeltas.size() -1;
-      for (int i = this.deltaAppliedIndex; i < listOfDeltas.size(); i++) {
-        Object o = listOfDeltas.get(i).apply(putEvent);
-        if(i < last) { 
-          putEvent.setOldValue(o);
-        }else {
-          putEvent.setNewValue(o);
-        }
-      }
-      return putEvent.getNewValue();
-    }
-    else {
-      throw new InternalGemFireException(
-          "ListOfDeltas.apply: putEvent is either null "
-              + "or is not of type EntryEventImpl");
-    }
-  }
-
-
-  public Object merge(Object toMerge, boolean isCreate)
-  {
-    throw new UnsupportedOperationException("Invocation not expected");
-  }
-  
-  public Object merge(Object toMerge)
-  {
-    this.listOfDeltas.add((Delta)toMerge); 
-    return this;
-  }
-  
-  public Object getResultantValue()
-  {
-    return this;
-  } 
-  
-  public int getNumDeltas() {
-    return this.listOfDeltas.size();
-  }
-  
-  public void setDeltaAppliedIndex(int deltaApplied) {
-    this.deltaAppliedIndex = deltaApplied;
-  }
-  
-  public List<Delta> getListOfDeltas() {
-    return Collections.unmodifiableList(this.listOfDeltas);
-  }
-}
-//SqlFabric changes END

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
index 6b664fe..8b9664f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
@@ -36,7 +36,6 @@ import com.gemstone.gemfire.cache.query.*;
 import com.gemstone.gemfire.cache.query.internal.DefaultQuery;
 import com.gemstone.gemfire.cache.query.internal.DefaultQueryService;
 import com.gemstone.gemfire.cache.query.internal.ExecutionContext;
-import com.gemstone.gemfire.cache.query.internal.IndexUpdater;
 import com.gemstone.gemfire.cache.query.internal.cq.CqService;
 import com.gemstone.gemfire.cache.query.internal.index.IndexCreationData;
 import com.gemstone.gemfire.cache.query.internal.index.IndexManager;
@@ -428,26 +427,6 @@ public class LocalRegion extends AbstractRegion
     return initializingRegion.get();
   }
 
-  /**
-   * Return true if the keys of this region implement
-   * {@link KeyWithRegionContext} that require region specific context
-   * initialization after deserialization or recovery from disk.
-   * 
-   * Currently used by SQLFabric for the optimized
-   * <code>CompactCompositeRegionKey</code> that points to the raw row bytes and
-   * so requires a handle to table schema for interpretation of those bytes.
-   */
-  public boolean keyRequiresRegionContext() {
-    return this.keyRequiresRegionContext;
-  }
-
-  /**
-   * Set the {@link #keyRequiresRegionContext} flag to given value.
-   */
-  public final void setKeyRequiresRegionContext(boolean v) {
-    this.keyRequiresRegionContext = v;
-  }
-
   public CancelCriterion getCancelCriterion() {
     return this.stopper;
   }
@@ -502,11 +481,9 @@ public class LocalRegion extends AbstractRegion
     this.initializationLatchAfterGetInitialImage = new StoppableCountDownLatch(this.stopper, 1);
     this.afterRegionCreateEventLatch = new StoppableCountDownLatch(this.stopper, 1);
 
-    // set the user-attribute object upfront for SQLFabric
     if (internalRegionArgs.getUserAttribute() != null) {
       setUserAttribute(internalRegionArgs.getUserAttribute());
     }
-    setKeyRequiresRegionContext(internalRegionArgs.keyRequiresRegionContext());
     initializingRegion.set(this);
 
     if (internalRegionArgs.getCachePerfStatsHolder() != null) {
@@ -674,10 +651,6 @@ public class LocalRegion extends AbstractRegion
     }
   }
 
-  public IndexUpdater getIndexUpdater() {
-    return this.entries.getIndexUpdater();
-  }
-
   boolean isCacheClosing()
   {
     return this.cache.isClosed();
@@ -854,10 +827,7 @@ public class LocalRegion extends AbstractRegion
                 && internalRegionArgs.isUsedForPartitionedRegionBucket()) {
               final PartitionedRegion pr = internalRegionArgs
                   .getPartitionedRegion();
-              internalRegionArgs.setIndexUpdater(pr.getIndexUpdater());
               internalRegionArgs.setUserAttribute(pr.getUserAttribute());
-              internalRegionArgs.setKeyRequiresRegionContext(pr
-                  .keyRequiresRegionContext());
               if (pr.isShadowPR()) {
                 newRegion = new BucketRegionQueue(subregionName, regionAttributes,
                   this, this.cache, internalRegionArgs);
@@ -1016,8 +986,6 @@ public class LocalRegion extends AbstractRegion
       }
   }
 
-  // split into a separate newCreateEntryEvent since SQLFabric may need to
-  // manipulate event before doing the put (e.g. posDup flag)
   @Retained
   public final EntryEventImpl newCreateEntryEvent(Object key, Object value,
       Object aCallbackArgument) {
@@ -1076,8 +1044,6 @@ public class LocalRegion extends AbstractRegion
       }
   }
 
-  // split into a separate newDestroyEntryEvent since SQLFabric may need to
-  // manipulate event before doing the put (e.g. posDup flag)
   @Retained
   public final EntryEventImpl newDestroyEntryEvent(Object key,
       Object aCallbackArgument) {
@@ -1588,15 +1554,8 @@ public class LocalRegion extends AbstractRegion
         event.setNewEventId(cache.getDistributedSystem());
       }
       Object oldValue = null;
-      // Sqlf changes begin
-      // see #40294.
-
-      // Rahul: this has to be an update.
-      // so executing it as an update.
-      boolean forceUpdateForDelta = event.hasDelta();
-      // Sqlf Changes end.
       if (basicPut(event, false, // ifNew
-          forceUpdateForDelta, // ifOld
+          false, // ifOld
           null, // expectedOldValue
           false // requireOldValue
       )) {
@@ -1612,8 +1571,6 @@ public class LocalRegion extends AbstractRegion
       return handleNotAvailable(oldValue);
   }
 
-  // split into a separate newUpdateEntryEvent since SQLFabric may need to
-  // manipulate event before doing the put (e.g. posDup flag)
   @Retained
   public final EntryEventImpl newUpdateEntryEvent(Object key, Object value,
       Object aCallbackArgument) {
@@ -1643,17 +1600,6 @@ public class LocalRegion extends AbstractRegion
       if (!eventReturned) event.release();
     }
   }
-  /**
-   * Creates an EntryEventImpl that is optimized to not fetch data from HDFS.
-   * This is meant to be used by PUT dml from GemFireXD.
-   */
-  @Retained
-  public final EntryEventImpl newPutEntryEvent(Object key, Object value,
-      Object aCallbackArgument) {
-    EntryEventImpl ev = newUpdateEntryEvent(key, value, aCallbackArgument);
-    ev.setPutDML(true);
-    return ev;
-  }
   private void extractDeltaIntoEvent(Object value, EntryEventImpl event) {
     // 1. Check for DS-level delta property.
     // 2. Default value for operation type is UPDATE, so no need to check that here.
@@ -3579,8 +3525,6 @@ public class LocalRegion extends AbstractRegion
    * Returns the value of the entry with the given key as it is stored on disk.
    * While the value may be read from disk, it is <b>not </b> stored into the
    * entry in the VM. This method is intended for testing purposes only.
-   * DO NOT use in product code else it will break SQLFabric that has cases
-   * where routing object is not part of only the key.
    *
    * @throws EntryNotFoundException
    *           No entry with <code>key</code> exists
@@ -3619,8 +3563,7 @@ public class LocalRegion extends AbstractRegion
   /**
    * Get the serialized bytes from disk. This method only looks for the value on
    * the disk, ignoring heap data. This method is intended for testing purposes
-   * only. DO NOT use in product code else it will break SQLFabric that has
-   * cases where routing object is not part of only the key.
+   * only. 
    * 
    * @param key the object whose hashCode is used to find the value
    * @return either a byte array, a CacheDeserializable with the serialized value,
@@ -3675,9 +3618,6 @@ public class LocalRegion extends AbstractRegion
   /**
    * Does a get that attempts to not fault values in from disk or make the entry
    * the most recent in the LRU.
-   * 
-   * Originally implemented in WAN gateway code and moved here in the sqlfire
-   * "cheetah" branch.
    * @param adamant fault in and affect LRU as a last resort
    * @param allowTombstone also return Token.TOMBSTONE if the entry is deleted
    * @param serializedFormOkay if the serialized form can be returned
@@ -5069,9 +5009,6 @@ public class LocalRegion extends AbstractRegion
   
   /**
    * Get the best iterator for the region entries.
-   * 
-   * TODO there has been some work on this on the sqlfire branch that should
-   * be picked up here.
    */
   public Iterator<RegionEntry> getBestIterator(boolean includeValues) {
     if(this instanceof DistributedRegion) {
@@ -5395,12 +5332,6 @@ public class LocalRegion extends AbstractRegion
         callbackArg = new GatewaySenderEventCallbackArgument(callbackArg);
       }
     }
-    //Asif: Modified the call to this constructor by passing the new value obtained from remote site
-    //instead of null .
-    //The need for this arose, because creation of EntryEvent, makes call to PartitionResolver,
-    //to get Hash. If the partitioning column is different from primary key, 
-    //the resolver for Sqlfabric is not able to obtain the hash object used for creation of KeyInfo  
-     
     @Released final EntryEventImpl event = EntryEventImpl.create(this, Operation.CREATE, key,
        value, callbackArg,  false /* origin remote */, client.getDistributedMember(),
         true /* generateCallbacks */,
@@ -5420,9 +5351,6 @@ public class LocalRegion extends AbstractRegion
     }
 
     // Set the new value to the input byte[] if it isn't null
-    /// For SqlFabric, if the new value happens to be an serialized object, then 
-    //it needs to be converted into VMCachedDeserializable , or serializable delta 
-    // as the case may be
     if (value != null) {
       // If the byte[] represents an object, then store it serialized
       // in a CachedDeserializable; otherwise store it directly as a byte[]
@@ -6064,12 +5992,6 @@ public class LocalRegion extends AbstractRegion
     long lastModifiedTime = event.getEventTime(lastModified);
     updateStatsForPut(entry, lastModifiedTime, lruRecentUse);
     if (!isProxy()) {
-      //if (this.isUsedForPartitionedRegionBucket) {
-      //  if (this.sqlfIndexManager != null) {
-      //    this.sqlfIndexManager.onEvent(this, event, entry);
-      //  }
-      //}
-
       if (!clearConflict && this.indexManager != null) {
         try {
           if (!entry.isInvalid()) {
@@ -6340,7 +6262,6 @@ public class LocalRegion extends AbstractRegion
          }
        }
        isDup = this.eventTracker.hasSeenEvent(event);
-       // don't clobber existing posDup flag e.g. set from SQLFabric client
        if (isDup) {
          event.setPossibleDuplicate(true);
          if (this.concurrencyChecksEnabled && event.getVersionTag() == null) {
@@ -7844,25 +7765,9 @@ public class LocalRegion extends AbstractRegion
       }
     }
   }
-  void cleanUpOnIncompleteOp(EntryEventImpl event,   RegionEntry re, 
-      boolean eventRecorded, boolean updateStats, boolean isReplace) {
-    //TODO:Asif: This is incorrect implementation for replicated region in case of
-    //sql fabric, as sqlf index would already be  updated, if eventRecorded 
-    //flag is true.So if entry is being removed , 
-    //then the sqlfindex also needs to be corrected
-    IndexUpdater iu = this.getIndexUpdater(); // sqlf system
-    if(!eventRecorded || iu ==null || isReplace) {
-    //Ok to remove entry whether sqlfabric or gfe as index has not been modified yet by the operation
-      this.entries.removeEntry(event.getKey(), re, updateStats) ;      
-    }else {
-      // a sqlf system, with event recorded as true. we need to update index.
-      //Use the current event to indicate destroy.should be ok
-      Operation oldOp = event.getOperation();
-      event.setOperation(Operation.DESTROY);
-      this.entries.removeEntry(event.getKey(), re, updateStats, event, this, iu);
-      event.setOperation(oldOp);
-    } 
-    
+  void cleanUpOnIncompleteOp(EntryEventImpl event, RegionEntry re) {
+    //Ok to remove entry as index has not been modified yet by the operation
+    this.entries.removeEntry(event.getKey(), re, false) ;      
   }
 
   static void validateRegionName(String name)
@@ -10531,8 +10436,6 @@ public class LocalRegion extends AbstractRegion
   }
 
 
-  // split into a separate newPutAllOperation since SQLFabric may need to
-  // manipulate event before doing the put (e.g. posDup flag)
   public final DistributedPutAllOperation newPutAllOperation(Map<?, ?> map, Object callbackArg) {
     if (map == null) {
       throw new NullPointerException(LocalizedStrings
@@ -10556,12 +10459,6 @@ public class LocalRegion extends AbstractRegion
     DistributedPutAllOperation dpao = new DistributedPutAllOperation(event, map.size(), false);
     return dpao;
   }
-    public final DistributedPutAllOperation newPutAllForPUTDmlOperation(Map<?, ?> map, Object callbackArg) {
-    DistributedPutAllOperation dpao = newPutAllOperation(map, callbackArg);
-    dpao.getEvent().setPutDML(true);
-    return dpao;
-  }
-
   
   public final DistributedRemoveAllOperation newRemoveAllOperation(Collection<?> keys, Object callbackArg) {
     if (keys == null) {
@@ -10613,8 +10510,6 @@ public class LocalRegion extends AbstractRegion
         putallOp, this, Operation.PUTALL_CREATE, key, value);
 
     try {
-    event.setPutDML(putallOp.getEvent().isPutDML());
-    
     if (tagHolder != null) {
       event.setVersionTag(tagHolder.getVersionTag());
       event.setFromServer(tagHolder.isFromServer());
@@ -11015,8 +10910,7 @@ public class LocalRegion extends AbstractRegion
       final CacheProfile prof = (CacheProfile)profile;
 
       // if region in cache is not yet initialized, exclude
-      if (prof.regionInitialized // fix for bug 41102
-          && !prof.memberUnInitialized) {
+      if (prof.regionInitialized) { // fix for bug 41102
         // cut the visit short if we find a CacheLoader
         return !prof.hasCacheLoader;
       }
@@ -11033,8 +10927,8 @@ public class LocalRegion extends AbstractRegion
       assert profile instanceof CacheProfile;
       final CacheProfile prof = (CacheProfile)profile;
 
-      // if region in cache is in recovery, or member not initialized exclude
-      if (!prof.inRecovery && !prof.memberUnInitialized) {
+      // if region in cache is in recovery
+      if (!prof.inRecovery) {
         // cut the visit short if we find a CacheWriter
         return !prof.hasCacheWriter;
       }
@@ -11486,15 +11380,6 @@ public class LocalRegion extends AbstractRegion
     distributeUpdatedProfileOnSenderCreation();
   }
   
-  /**
-   * @since GemFire SqlFabric
-   *
-   */
-  void distributeUpdatedProfileOnHubCreation()
-  {
-    // No op
-  }  
-  
   void distributeUpdatedProfileOnSenderCreation()
   {
     // No op

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
index 4c04054..59b0893 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/Oplog.java
@@ -1867,7 +1867,6 @@ public final class Oplog implements CompactableOplog, Flushable {
     CountingDataInputStream dis = null;
     try {
       final LocalRegion currentRegion = LocalRegion.getInitializingRegion();
-      final boolean keyRequiresRegionContext = currentRegion != null ? currentRegion.keyRequiresRegionContext() : false;
       final Version version = getProductVersionIfOld();
       final ByteArrayDataInput in = new ByteArrayDataInput();
       final HeapDataOutputStream hdos = new HeapDataOutputStream(Version.CURRENT);
@@ -1909,7 +1908,7 @@ public final class Oplog implements CompactableOplog, Flushable {
           }
             break;
           case OPLOG_NEW_ENTRY_0ID:
-            readNewEntry(dis, opCode, deletedIds, recoverValues, currentRegion, keyRequiresRegionContext, version, in, hdos);
+            readNewEntry(dis, opCode, deletedIds, recoverValues, currentRegion, version, in, hdos);
             recordCount++;
             break;
           case OPLOG_MOD_ENTRY_1ID:
@@ -1920,7 +1919,7 @@ public final class Oplog implements CompactableOplog, Flushable {
           case OPLOG_MOD_ENTRY_6ID:
           case OPLOG_MOD_ENTRY_7ID:
           case OPLOG_MOD_ENTRY_8ID:
-            readModifyEntry(dis, opCode, deletedIds, recoverValues, currentRegion, keyRequiresRegionContext, version, in, hdos);
+            readModifyEntry(dis, opCode, deletedIds, recoverValues, currentRegion, version, in, hdos);
             recordCount++;
             break;
           case OPLOG_MOD_ENTRY_WITH_KEY_1ID:
@@ -1931,7 +1930,7 @@ public final class Oplog implements CompactableOplog, Flushable {
           case OPLOG_MOD_ENTRY_WITH_KEY_6ID:
           case OPLOG_MOD_ENTRY_WITH_KEY_7ID:
           case OPLOG_MOD_ENTRY_WITH_KEY_8ID:
-            readModifyEntryWithKey(dis, opCode, deletedIds, recoverValues, currentRegion, keyRequiresRegionContext, version, in,
+            readModifyEntryWithKey(dis, opCode, deletedIds, recoverValues, currentRegion, version, in,
                 hdos);
             recordCount++;
             break;
@@ -2414,7 +2413,7 @@ public final class Oplog implements CompactableOplog, Flushable {
    * @throws IOException
    */
   private void readNewEntry(CountingDataInputStream dis, byte opcode, OplogEntryIdSet deletedIds, boolean recoverValue,
-      final LocalRegion currentRegion, boolean keyRequiresRegionContext, Version version, ByteArrayDataInput in,
+      final LocalRegion currentRegion, Version version, ByteArrayDataInput in,
       HeapDataOutputStream hdos) throws IOException {
     final boolean isPersistRecoveryDebugEnabled = logger.isTraceEnabled(LogMarker.PERSIST_RECOVERY);
     
@@ -2553,9 +2552,6 @@ public final class Oplog implements CompactableOplog, Flushable {
           }
         } else {
           Object key = deserializeKey(keyBytes, version, in);
-          if (keyRequiresRegionContext) {
-            ((KeyWithRegionContext) key).setRegionContext(currentRegion);
-          }
           {
             Object oldValue = getRecoveryMap().put(oplogKeyId, key);
             if (oldValue != null) {
@@ -2605,7 +2601,7 @@ public final class Oplog implements CompactableOplog, Flushable {
    * @throws IOException
    */
   private void readModifyEntry(CountingDataInputStream dis, byte opcode, OplogEntryIdSet deletedIds, boolean recoverValue,
-      LocalRegion currentRegion, boolean keyRequiresRegionContext, Version version, ByteArrayDataInput in, HeapDataOutputStream hdos)
+      LocalRegion currentRegion, Version version, ByteArrayDataInput in, HeapDataOutputStream hdos)
       throws IOException {
     final boolean isPersistRecoveryDebugEnabled = logger.isTraceEnabled(LogMarker.PERSIST_RECOVERY);
     
@@ -2711,9 +2707,6 @@ public final class Oplog implements CompactableOplog, Flushable {
         byte[] keyBytes = (byte[]) skippedKeyBytes.get(oplogKeyId);
         if (keyBytes != null) {
           key = deserializeKey(keyBytes, version, in);
-          if (keyRequiresRegionContext) {
-            ((KeyWithRegionContext) key).setRegionContext(currentRegion);
-          }
         }
       }
       if (isPersistRecoveryDebugEnabled) {
@@ -2829,7 +2822,7 @@ public final class Oplog implements CompactableOplog, Flushable {
    * @throws IOException
    */
   private void readModifyEntryWithKey(CountingDataInputStream dis, byte opcode, OplogEntryIdSet deletedIds, boolean recoverValue,
-      final LocalRegion currentRegion, final boolean keyRequiresRegionContext, Version version, ByteArrayDataInput in,
+      final LocalRegion currentRegion, Version version, ByteArrayDataInput in,
       HeapDataOutputStream hdos) throws IOException {
     long oplogOffset = -1;
 
@@ -2966,9 +2959,6 @@ public final class Oplog implements CompactableOplog, Flushable {
         }
       } else {
         Object key = deserializeKey(keyBytes, version, in);
-        if (keyRequiresRegionContext) {
-          ((KeyWithRegionContext) key).setRegionContext(currentRegion);
-        }
         Object oldValue = getRecoveryMap().put(oplogKeyId, key);
         if (oldValue != null) {
           throw new AssertionError(LocalizedStrings.Oplog_DUPLICATE_CREATE.toLocalizedString(oplogKeyId));
@@ -7039,8 +7029,6 @@ public final class Oplog implements CompactableOplog, Flushable {
     public void handleValueOverflow(RegionEntryContext context) {throw new IllegalStateException();}
 
     @Override
-    public void afterValueOverflow(RegionEntryContext context) {throw new IllegalStateException();}
-    @Override
     public Object prepareValueForCache(RegionEntryContext r, Object val, boolean isEntryUpdate) { throw new IllegalStateException("Should never be called");  }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PRHARedundancyProvider.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PRHARedundancyProvider.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PRHARedundancyProvider.java
index f933024..c33efb7 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PRHARedundancyProvider.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PRHARedundancyProvider.java
@@ -432,8 +432,6 @@ public class PRHARedundancyProvider
       this.prRegion.checkReadiness();
       Set<InternalDistributedMember> available = this.prRegion
           .getRegionAdvisor().adviseInitializedDataStore();
-      // remove uninitialized members for bucket creation
-      this.prRegion.getCache().removeUnInitializedMembers(available);
       InternalDistributedMember target = null;
       available.removeAll(attempted);
       for (InternalDistributedMember member : available) {
@@ -575,8 +573,6 @@ public class PRHARedundancyProvider
         // Always go back to the advisor, see if any fresh data stores are
           // present.
         Set<InternalDistributedMember> allStores = getAllStores(partitionName);
-        // remove nodes that are not fully initialized
-        this.prRegion.getCache().removeUnInitializedMembers(allStores);
 
         loggedInsufficentStores  = checkSufficientStores(allStores, 
             loggedInsufficentStores);
@@ -776,7 +772,6 @@ public class PRHARedundancyProvider
       //  the parent's in case of colocation) so it is now passed
       //InternalDistributedMember targetPrimary = getPreferredDataStore(
       //    acceptedMembers, Collections.<InternalDistributedMember> emptySet());
-      this.prRegion.getCache().removeUnInitializedMembers(acceptedMembers);
       targetPrimary = getPreferredDataStore(acceptedMembers, Collections
           .<InternalDistributedMember> emptySet());
     }
@@ -1580,9 +1575,6 @@ public class PRHARedundancyProvider
     if (!PRHARedundancyProvider.this.prRegion.isDataStore()) {
       return;
     }
-    if (cache.isUnInitializedMember(cache.getMyId())) {
-      return;
-    }
     Runnable task = new RecoveryRunnable(this) {
       @Override
       public void run2()

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionAttributesImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionAttributesImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionAttributesImpl.java
index 47548f3..dd90a62 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionAttributesImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionAttributesImpl.java
@@ -384,16 +384,6 @@ public class PartitionAttributesImpl implements PartitionAttributes,
       .append("]") .toString();
   }
 
-  public String getStringForSQLF() {
-    final StringBuilder sb = new StringBuilder();
-    return sb.append("redundantCopies=").append(getRedundantCopies()).append(
-        ",totalMaxMemory=").append(this.totalMaxMemory).append(
-        ",totalNumBuckets=").append(this.totalNumBuckets).append(
-        ",colocatedWith=").append(this.colocatedRegionName).append(
-        ",recoveryDelay=").append(this.recoveryDelay).append(
-        ",startupRecoveryDelay=").append(this.startupRecoveryDelay).toString();
-  }
-
   /**
    * @throws IllegalStateException if off-heap and the actual value is not yet known (because the DistributedSystem has not yet been created)
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
index 9375d04..26c91e0 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegion.java
@@ -1035,16 +1035,6 @@ public class PartitionedRegion extends LocalRegion implements
   }
   
   @Override
-  void distributeUpdatedProfileOnHubCreation()
-  {
-    if (!(this.isClosed || this.isLocallyDestroyed)) {
-      // tell others of the change in status
-      this.requiresNotification = true;
-      new UpdateAttributesProcessor(this).distribute(false);      
-    }
-  }
-
-  @Override
   void distributeUpdatedProfileOnSenderCreation()
   {
     if (!(this.isClosed || this.isLocallyDestroyed)) {
@@ -1376,11 +1366,7 @@ public class PartitionedRegion extends LocalRegion implements
     boolean colocatedLockAcquired = false;
     try {
       boolean colocationComplete = false;
-      if (colocatedRegion != null && !prConfig.isColocationComplete() &&
-        // if the current node is marked uninitialized (SQLF DDL replay in
-        // progress) then colocation will definitely not be marked complete so
-        // avoid taking the expensive region lock
-          !getCache().isUnInitializedMember(getDistributionManager().getId())) {
+      if (colocatedRegion != null && !prConfig.isColocationComplete()) {
         colocatedLock = colocatedRegion.getRegionLock();
         colocatedLock.lock();
         colocatedLockAcquired = true;
@@ -1389,16 +1375,7 @@ public class PartitionedRegion extends LocalRegion implements
         if (parentConf.isColocationComplete()
             && parentConf.hasSameDataStoreMembers(prConfig)) {
           colocationComplete = true;
-          // check if all the nodes have been initialized (SQLF bug #42089)
-          for (Node node : nodes) {
-            if (getCache().isUnInitializedMember(node.getMemberId())) {
-              colocationComplete = false;
-              break;
-            }
-          }
-          if (colocationComplete) {
-            prConfig.setColocationComplete();
-          }
+          prConfig.setColocationComplete();
         }
       }
 
@@ -1935,13 +1912,6 @@ public class PartitionedRegion extends LocalRegion implements
       if (targetNode == null) {
         try {
           bucketStorageAssigned=false;
-          // if this is a Delta update, then throw exception since the key doesn't
-          // exist if there is no bucket for it yet
-          if (event.hasDelta()) {
-            throw new EntryNotFoundException(LocalizedStrings.
-              PartitionedRegion_CANNOT_APPLY_A_DELTA_WITHOUT_EXISTING_ENTRY
-                .toLocalizedString());
-          }
           targetNode = createBucket(bucketId.intValue(), event.getNewValSizeForPR(),
               null);
         }
@@ -3462,10 +3432,8 @@ public class PartitionedRegion extends LocalRegion implements
       boolean isBucketSetAsFilter) {
     final Set routingKeys = execution.getFilter();
     final boolean primaryMembersNeeded = function.optimizeForWrite();
-    final boolean hasRoutingObjects = execution.hasRoutingObjects();
     HashMap<Integer, HashSet> bucketToKeysMap = FunctionExecutionNodePruner
-        .groupByBucket(this, routingKeys, primaryMembersNeeded,
-            hasRoutingObjects, isBucketSetAsFilter);
+        .groupByBucket(this, routingKeys, primaryMembersNeeded, false, isBucketSetAsFilter);
     HashMap<InternalDistributedMember, HashSet> memberToKeysMap = new HashMap<InternalDistributedMember, HashSet>();
     HashMap<InternalDistributedMember, HashSet<Integer>> memberToBuckets = FunctionExecutionNodePruner
     .groupByMemberToBuckets(this, bucketToKeysMap.keySet(), primaryMembersNeeded);    
@@ -3555,7 +3523,7 @@ public class PartitionedRegion extends LocalRegion implements
     else {
       localBucketSet = FunctionExecutionNodePruner
       .getBucketSet(PartitionedRegion.this, localKeys,
-                    hasRoutingObjects, isBucketSetAsFilter);
+                    false, isBucketSetAsFilter);
       
       remoteOnly = false;
     }
@@ -3591,7 +3559,7 @@ public class PartitionedRegion extends LocalRegion implements
         FunctionRemoteContext context = new FunctionRemoteContext(function,
             execution.getArgumentsForMember(recip.getId()), memKeys,
             FunctionExecutionNodePruner.getBucketSet(this, memKeys,
-                hasRoutingObjects, isBucketSetAsFilter), execution.isReExecute(),
+                false, isBucketSetAsFilter), execution.isReExecute(),
                 execution.isFnSerializationReqd());
         recipMap.put(recip, context);
       }
@@ -3621,15 +3589,8 @@ public class PartitionedRegion extends LocalRegion implements
     if (isBucketSetAsFilter) {
       bucketId = ((Integer) key).intValue();
     } else {
-      if (execution.hasRoutingObjects()) {
-        bucketId = Integer.valueOf(PartitionedRegionHelper
-            .getHashKey(this, key));
-      } else {
-        // bucketId = Integer.valueOf(PartitionedRegionHelper.getHashKey(this,
-        // Operation.FUNCTION_EXECUTION, key, null));
-        bucketId = Integer.valueOf(PartitionedRegionHelper.getHashKey(this,
+      bucketId = Integer.valueOf(PartitionedRegionHelper.getHashKey(this,
             Operation.FUNCTION_EXECUTION, key, null, null));
-      }
     }
     InternalDistributedMember targetNode = null;
     if (function.optimizeForWrite()) {
@@ -5066,21 +5027,6 @@ public class PartitionedRegion extends LocalRegion implements
   /**
    * generates new partitioned region ID globally.
    */
-  // !!!:ezoerner:20080321 made this function public and static.
-  // @todo should be moved to the Distributed System level as a general service
-  // for getting a unique id, with different "domains" for different
-  // contexts
-  // :soubhik:pr_func merge20914:21056: overloaded static and non-static version of generatePRId.
-  //   static version is used mainly with sqlf & non-static in gfe.
-  public static int generatePRId(InternalDistributedSystem sys, Cache cache) {
-    
-    GemFireCacheImpl gfcache = (GemFireCacheImpl) cache;
-    
-    if(gfcache == null) return 0;
-    
-    return _generatePRId(sys, gfcache.getPartitionedRegionLockService());
-  }
-  
   public int generatePRId(InternalDistributedSystem sys) {
     final DistributedLockService lockService = getPartitionedRegionLockService();
     return _generatePRId(sys, lockService);
@@ -6257,15 +6203,6 @@ public class PartitionedRegion extends LocalRegion implements
   }
 
   /**
-   * Currently used by SQLFabric to get a non-wrapped iterator for all entries
-   * for index consistency check.
-   */
-  public Set allEntries() {
-    return new PREntriesSet();
-  }
-
-
-  /**
    * Set view of entries. This currently extends the keySet iterator and
    * performs individual getEntry() operations using the keys
    * 
@@ -7678,20 +7615,7 @@ public class PartitionedRegion extends LocalRegion implements
   }
         
   @Override
-  public void localDestroyRegion(Object aCallbackArgument) {
-    localDestroyRegion(aCallbackArgument, false);
-  }
-
-  /**
-   * Locally destroy a region.
-   * 
-   * SQLFabric change: The parameter "ignoreParent" has been added to allow
-   * skipping the check for parent colocated region. This is because SQLFabric
-   * DDLs are distributed in any case and are guaranteed to be atomic (i.e. no
-   * concurrent DMLs on that table). Without this it is quite ugly to implement
-   * "TRUNCATE TABLE" which first drops the table and recreates it.
-   */
-  public void localDestroyRegion(Object aCallbackArgument, boolean ignoreParent)
+  public void localDestroyRegion(Object aCallbackArgument)
   {
     getDataView().checkSupportsRegionDestroy();
     String prName = this.getColocatedWith();
@@ -7707,7 +7631,7 @@ public class PartitionedRegion extends LocalRegion implements
       }
     }
 
-    if ((!ignoreParent && prName != null)
+    if ((prName != null)
         || (!childRegionsWithoutSendersList.isEmpty())) {
       throw new UnsupportedOperationException(
           "Any Region in colocation chain cannot be destroyed locally.");
@@ -9430,8 +9354,6 @@ public class PartitionedRegion extends LocalRegion implements
 
   /**
    * This method is intended for testing purposes only.
-   * DO NOT use in product code else it will break SQLFabric that has cases
-   * where routing object is not part of only the key.
    */
   @Override
   public Object getValueOnDisk(Object key) throws EntryNotFoundException {
@@ -9444,8 +9366,6 @@ public class PartitionedRegion extends LocalRegion implements
   
   /**
    * This method is intended for testing purposes only.
-   * DO NOT use in product code else it will break SQLFabric that has cases
-   * where routing object is not part of only the key.
    */
   @Override
   public Object getValueOnDiskOrBuffer(Object key) throws EntryNotFoundException {
@@ -9565,33 +9485,13 @@ public class PartitionedRegion extends LocalRegion implements
   }
 
   public PartitionResolver getPartitionResolver() {
-    // [SQLFabric] use PartitionAttributes to get the the resolver
-    // since it may change after ALTER TABLE
     return this.partitionAttributes.getPartitionResolver();
   }
 
   public String getColocatedWith() {
-    // [SQLFabric] use PartitionAttributes to get colocated region
-    // since it may change after ALTER TABLE
     return this.partitionAttributes.getColocatedWith();
   }
 
-  // For SQLFabric ALTER TABLE. Need to set the colocated region using
-  // PartitionAttributesImpl and also reset the parentAdvisor for
-  // BucketAdvisors.
-  /**
-   * Set the colocated with region path and adjust the BucketAdvisor's. This
-   * should *only* be invoked when region is just newly created and has no data
-   * or existing buckets else will have undefined behaviour.
-   * 
-   * @since GemFire 6.5
-   */
-  public void setColocatedWith(String colocatedRegionFullPath) {
-    ((PartitionAttributesImpl)this.partitionAttributes)
-        .setColocatedWith(colocatedRegionFullPath);
-    this.getRegionAdvisor().resetBucketAdvisorParents();
-  }
-
   /**
    * Used to get membership events from our advisor to implement
    * RegionMembershipListener invocations. This is copied almost in whole from
@@ -9649,98 +9549,6 @@ public class PartitionedRegion extends LocalRegion implements
     }
   }
   
-  /*
-   * This is an internal API for sqlFabric only <br>
-   * This is usefull to execute a function on set of nodes irrelevant of the
-   * routinKeys <br>
-   * notes : This API uses DefaultResultCollector. If you want your Custome
-   * Result collector, let me know
-   * 
-   * @param functionName
-   * @param args
-   * @param nodes
-   *                Set of DistributedMembers on which this function will be
-   *                executed
-   * @throws Exception
-   *//*
-  public ResultCollector executeFunctionOnNodes(String functionName,
-      Serializable args, Set nodes) throws Exception {
-    Assert.assertTrue(functionName != null, "Error: functionName is null");
-    Assert.assertTrue(nodes != null, "Error: nodes set is null");
-    Assert.assertTrue(nodes.size() != 0, "Error: empty nodes Set");
-    ResultCollector rc = new DefaultResultCollector();
-    boolean isSelf = nodes.remove(getMyId());
-    PartitionedRegionFunctionResponse response = null;
-    //TODO Yogesh: this API is broken after Resultsender implementation
-    //response = new PartitionedRegionFunctionResponse(this.getSystem(), nodes,
-    //    rc);
-    Iterator i = nodes.iterator();
-    while (i.hasNext()) {
-      InternalDistributedMember recip = (InternalDistributedMember)i.next();
-      PartitionedRegionFunctionMessage.send(recip, this, functionName, args,
-          null routingKeys , response, null);
-    }
-    if (isSelf) {
-      // execute locally and collect the result
-      if (this.dataStore != null) {
-        this.dataStore.executeOnDataStore(
-            null routingKeys , functionName, args, 0,null,rc,null);
-      }
-    }
-    return response;
-  }*/
-
-
-  /*
-   * This is an internal API for sqlFabric only <br>
-   * API for invoking a function using primitive ints as the routing objects
-   * (i.e. passing the hashcodes of the routing objects directly). <br>
-   * notes : This API uses DefaultResultCollector. If you want to pass your
-   * Custom Result collector, let me know
-   * 
-   * @param functionName
-   * @param args
-   * @param hashcodes
-   *          hashcodes of the routing objects
-   * @throws Exception
-   *//*
-  public ResultCollector executeFunctionUsingHashCodes(String functionName,
-      Serializable args, int hashcodes[]) throws Exception {
-    Assert.assertTrue(functionName != null, "Error: functionName is null");
-    Assert.assertTrue(hashcodes != null, "Error: hashcodes array is null");
-    Assert.assertTrue(hashcodes.length != 0, "Error: empty hashcodes array");
-    Set nodes = new HashSet();
-    for (int i = 0; i < hashcodes.length; i++) {
-      int bucketId = hashcodes[i] % getTotalNumberOfBuckets();
-      InternalDistributedMember n = getNodeForBucketRead(bucketId);
-      nodes.add(n);
-    }
-    return executeFunctionOnNodes(functionName, args, nodes);
-  }*/
-
-  /**
-   * This is an internal API for sqlFabric only <br>
-   * Given a array of routing objects, returns a set of members on which the (owner of each
-   * buckets)
-   * 
-   * @param routingObjects array of routing objects passed 
-   * @return Set of  InternalDistributedMembers
-   */
-  public Set getMembersFromRoutingObjects(Object[] routingObjects) {
-    Assert.assertTrue(routingObjects != null, "Error: null routingObjects ");
-    Assert.assertTrue(routingObjects.length != 0, "Error: empty routingObjects ");
-    Set nodeSet = new HashSet();
-    int bucketId;
-    for (int i = 0; i < routingObjects.length; i++) {
-      bucketId = PartitionedRegionHelper.getHashKey(routingObjects[i],
-                                                    getTotalNumberOfBuckets());
-      InternalDistributedMember lnode = getOrCreateNodeForBucketRead(bucketId);
-      if (lnode != null) {
-        nodeSet.add(lnode);
-      }
-    }
-    return nodeSet;
-  }
   @Override
   protected RegionEntry basicGetTXEntry(KeyInfo keyInfo) {
     int bucketId = keyInfo.getBucketId();
@@ -10525,9 +10333,7 @@ public class PartitionedRegion extends LocalRegion implements
   }
 
   /**
-   * Returns the local BucketRegion given the key and value. Particularly useful
-   * for SQLFabric where the routing object may be part of value and determining
-   * from key alone will require an expensive global index lookup.
+   * Returns the local BucketRegion given the key and value.
    * Returns null if no BucketRegion exists.
    */
   public BucketRegion getBucketRegion(Object key, Object value) {
@@ -10754,74 +10560,6 @@ public class PartitionedRegion extends LocalRegion implements
     }  
   }
 
-  /**
-   * Clear local primary buckets.
-   * This is currently only used by gemfirexd truncate table
-   * to clear the partitioned region.
-   */
-  public void clearLocalPrimaries() {
- // rest of it should be done only if this is a store while RecoveryLock
-    // above still required even if this is an accessor
-    if (getLocalMaxMemory() > 0) {
-      // acquire the primary bucket locks
-      // do this in a loop to handle the corner cases where a primary
-      // bucket region ceases to be so when we actually take the lock
-      // (probably not required to do this in loop after the recovery lock)
-      // [sumedh] do we need both recovery lock and bucket locks?
-      boolean done = false;
-      Set<BucketRegion> lockedRegions = null;
-      while (!done) {
-        lockedRegions = getDataStore().getAllLocalPrimaryBucketRegions();
-        done = true;
-        for (BucketRegion br : lockedRegions) {
-          try {
-            br.doLockForPrimary(false);
-          } catch (RegionDestroyedException rde) {
-            done = false;
-            break;
-          } catch (PrimaryBucketException pbe) {
-            done = false;
-            break;
-          } catch (Exception e) {
-            // ignore any other exception
-            logger.debug(
-                "GemFireContainer#clear: ignoring exception "
-                    + "in bucket lock acquire", e);
-          }
-        }
-      }
-      
-      try {
-        // now clear the bucket regions; we go through the primary bucket
-        // regions so there is distribution for every bucket but that
-        // should be performant enough
-        for (BucketRegion br : lockedRegions) {
-          try {
-            br.clear();
-          } catch (Exception e) {
-            // ignore any other exception
-            logger.debug(
-                "GemFireContainer#clear: ignoring exception "
-                    + "in bucket clear", e);
-          }
-        }
-      } finally {
-        // release the bucket locks
-        for (BucketRegion br : lockedRegions) {
-          try {
-            br.doUnlockForPrimary();
-          } catch (Exception e) {
-            // ignore all exceptions at this stage
-            logger.debug(
-                "GemFireContainer#clear: ignoring exception "
-                    + "in bucket lock release", e);
-          }
-        }
-      }
-    }
-    
-  }
-
   public void shadowPRWaitForBucketRecovery() {
     assert this.isShadowPR();
     PartitionedRegion userPR = ColocationHelper.getLeaderRegion(this);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDataStore.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDataStore.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDataStore.java
index 3855adc..494c288 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDataStore.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionDataStore.java
@@ -24,7 +24,6 @@ import com.gemstone.gemfire.cache.execute.Function;
 import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.execute.ResultSender;
 import com.gemstone.gemfire.cache.query.QueryInvalidException;
-import com.gemstone.gemfire.cache.query.internal.IndexUpdater;
 import com.gemstone.gemfire.cache.query.internal.QCompiler;
 import com.gemstone.gemfire.cache.query.internal.index.IndexCreationData;
 import com.gemstone.gemfire.cache.query.internal.index.PartitionedIndex;
@@ -431,19 +430,7 @@ public class PartitionedRegionDataStore implements HasCachePerfStats
               Object redundancyLock = lockRedundancyLock(moveSource,
                   possiblyFreeBucketId, replaceOffineData);
               //DAN - I hope this is ok to do without that bucket admin lock
-              // Take SQLF lock to wait for any ongoing index initializations.
-              // The lock is taken here in addition to that in
-              // DistributedRegion#initialize() so as to release only after
-              // assignBucketRegion() has been invoked (see bug #41877).
-              // Assumes that the IndexUpdater#lockForGII() lock is re-entrant.
-              final IndexUpdater indexUpdater = this.partitionedRegion
-              .getIndexUpdater();
-              boolean sqlfIndexLocked = false;
               try {
-                if (indexUpdater != null) {
-                  indexUpdater.lockForGII();
-                  sqlfIndexLocked = true;
-                }
                 buk.initializePrimaryElector(creationRequestor);
                 if (getPartitionedRegion().getColocatedWith() == null) {
                   buk.getBucketAdvisor().setShadowBucketDestroyed(false);
@@ -476,9 +463,6 @@ public class PartitionedRegionDataStore implements HasCachePerfStats
                   }
                 }
               } finally {
-                if (sqlfIndexLocked) {
-                  indexUpdater.unlockForGII();
-                }
                 releaseRedundancyLock(redundancyLock);
                 if(bukReg == null) {
                   buk.clearPrimaryElector();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ProxyRegionMap.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ProxyRegionMap.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ProxyRegionMap.java
index ee8e0c8..55d11fc 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ProxyRegionMap.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ProxyRegionMap.java
@@ -30,7 +30,6 @@ import com.gemstone.gemfire.cache.EntryNotFoundException;
 import com.gemstone.gemfire.cache.Operation;
 import com.gemstone.gemfire.cache.TimeoutException;
 import com.gemstone.gemfire.cache.TransactionId;
-import com.gemstone.gemfire.cache.query.internal.IndexUpdater;
 import com.gemstone.gemfire.distributed.internal.DM;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.ByteArrayDataInput;
@@ -56,23 +55,10 @@ import com.gemstone.gemfire.internal.offheap.annotations.Released;
  */
 final class ProxyRegionMap implements RegionMap {
 
-  /** An internal Listener for index maintenance for SQLFabric. */
-  private final IndexUpdater indexUpdater;
-
   protected ProxyRegionMap(LocalRegion owner, Attributes attr,
       InternalRegionArguments internalRegionArgs) {
     this.owner = owner;
     this.attr = attr;
-    if (internalRegionArgs != null) {
-      this.indexUpdater = internalRegionArgs.getIndexUpdater();
-    }
-    else {
-      this.indexUpdater = null;
-    }
-  }
-
-  public final IndexUpdater getIndexUpdater() {
-    return this.indexUpdater;
   }
 
   /**
@@ -249,13 +235,6 @@ final class ProxyRegionMap implements RegionMap {
     lastModified = // fix for bug 40129
       this.owner.basicPutPart2(event, markerEntry, true,
         lastModified, false /*Clear conflict occurred */);
-    // invoke SQLFabric index manager if present
-    final IndexUpdater indexUpdater = getIndexUpdater();
-    if (indexUpdater != null) {
-      // postEvent not required to be invoked since this is currently used
-      // only for FK checks
-      indexUpdater.onEvent(this.owner, event, markerEntry);
-    }
     this.owner.basicPutPart3(event, markerEntry, true,
           lastModified, true, ifNew, ifOld, expectedOldValue, requireOldValue);
     return markerEntry;
@@ -399,7 +378,7 @@ final class ProxyRegionMap implements RegionMap {
   }
 
   public void removeEntry(Object key, RegionEntry re, boolean updateStat,
-      EntryEventImpl event, LocalRegion owner, IndexUpdater indexUpdater) {
+      EntryEventImpl event, LocalRegion owner) {
     // nothing to do
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/QueuedOperation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/QueuedOperation.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/QueuedOperation.java
index 22f9903..7be2bb9 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/QueuedOperation.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/QueuedOperation.java
@@ -161,13 +161,7 @@ public class QueuedOperation
       key = DataSerializer.readObject(in);
       if (op.isUpdate() || op.isCreate()) {
         deserializationPolicy = in.readByte();
-        if (deserializationPolicy ==
-            DistributedCacheOperation.DESERIALIZATION_POLICY_EAGER) {
-          valueObj = DataSerializer.readObject(in);
-        }
-        else {
-          value = DataSerializer.readByteArray(in);
-        }
+        value = DataSerializer.readByteArray(in);
       }
     }
     return new QueuedOperation(op, key, value, valueObj, deserializationPolicy,
@@ -183,13 +177,7 @@ public class QueuedOperation
       DataSerializer.writeObject(this.key, out);
       if (this.op.isUpdate() || this.op.isCreate()) {
         out.writeByte(this.deserializationPolicy);
-        if (this.deserializationPolicy !=
-            DistributedCacheOperation.DESERIALIZATION_POLICY_EAGER) {
-          DataSerializer.writeByteArray(this.value, out);
-        }
-        else {
-          DataSerializer.writeObject(this.valueObj, out);
-        }
+        DataSerializer.writeByteArray(this.value, out);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEntry.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEntry.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEntry.java
index b35eaa3..48ed5db 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEntry.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionEntry.java
@@ -379,16 +379,14 @@ public interface RegionEntry {
   /**
    * Gets the value for this entry. For DiskRegions, unlike
    * {@link #getValue(RegionEntryContext)} this will not fault in the value rather
-   * return a temporary copy. For SQLFabric this is used during table scans in
-   * queries when faulting in every value will be only an unnecessary overhead.
+   * return a temporary copy.
    */
   public Object getValueInVMOrDiskWithoutFaultIn(LocalRegion owner);
 
   /**
    * Gets the value for this entry. For DiskRegions, unlike
    * {@link #getValue(RegionEntryContext)} this will not fault in the value rather
-   * return a temporary copy. For SQLFabric this is used during table scans in
-   * queries when faulting in every value will be only an unnecessary overhead.
+   * return a temporary copy.
    * The value returned will be kept off heap (and compressed) if possible.
    */
   @Retained

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionMap.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionMap.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionMap.java
index a16f1ec..57f8853 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionMap.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionMap.java
@@ -27,7 +27,6 @@ import com.gemstone.gemfire.cache.EntryNotFoundException;
 import com.gemstone.gemfire.cache.Operation;
 import com.gemstone.gemfire.cache.TimeoutException;
 import com.gemstone.gemfire.cache.TransactionId;
-import com.gemstone.gemfire.cache.query.internal.IndexUpdater;
 import com.gemstone.gemfire.internal.cache.lru.LRUMapCallbacks;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
@@ -355,16 +354,13 @@ public interface RegionMap extends LRUMapCallbacks {
 
   /**
    * Removes the given key if the enclosing RegionEntry is still in this map for
-   * the given EntryEvent and updating the given {@link IndexUpdater} of the
-   * region ({@link #getIndexUpdater()}) for the event.
+   * the given EntryEvent
    */
   public void removeEntry(Object key, RegionEntry re, boolean updateStat,
-      EntryEventImpl event, LocalRegion owner, IndexUpdater indexUpdater);
+      EntryEventImpl event, LocalRegion owner);
 
   public void copyRecoveredEntries(RegionMap rm);
 
-  public IndexUpdater getIndexUpdater();
-  
   /**
    * Removes an entry that was previously destroyed and made into a tombstone.
    * 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteContainsKeyValueMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteContainsKeyValueMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteContainsKeyValueMessage.java
index a1b5d0c..d5a52d4 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteContainsKeyValueMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteContainsKeyValueMessage.java
@@ -123,9 +123,6 @@ public final class RemoteContainsKeyValueMessage extends RemoteOperationMessageW
       r.waitOnInitialization(); // bug #43371 - accessing a region before it's initialized
     }
 
-    if (r.keyRequiresRegionContext()) {
-      ((KeyWithRegionContext)this.key).setRegionContext(r);
-    }
     final boolean replyVal;
         if (this.valueCheck) {
           replyVal = r.containsValueForKey(this.key);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
index 822b317..2bec70f 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteDestroyMessage.java
@@ -357,9 +357,6 @@ public class RemoteDestroyMessage extends RemoteOperationMessageWithDirectReply
     if (eventSender == null) {
        eventSender = getSender();
     }
-    if (r.keyRequiresRegionContext()) {
-      ((KeyWithRegionContext)this.key).setRegionContext(r);
-    }
     @Released EntryEventImpl event = null;
     try {
     if (this.bridgeContext != null) {
@@ -468,8 +465,6 @@ public class RemoteDestroyMessage extends RemoteOperationMessageWithDirectReply
     if (this.hasOldValue){
       //out.writeBoolean(this.hasOldValue);
       // below boolean is not strictly required, but this is for compatibility
-      // with SQLFire code which writes as byte here to indicate whether
-      // oldValue is an object, serialized object or byte[]
       in.readByte();
       setOldValBytes(DataSerializer.readByteArray(in));
     }
@@ -595,12 +590,8 @@ public class RemoteDestroyMessage extends RemoteOperationMessageWithDirectReply
   
   private void setOldValueIsSerialized(boolean isSerialized) {
     if (isSerialized) {
-      if (CachedDeserializableFactory.preferObject()) {
-        this.oldValueIsSerialized = true; //VALUE_IS_OBJECT;
-      } else {
-        // Defer serialization until toData is called.
-        this.oldValueIsSerialized = true; //VALUE_IS_SERIALIZED_OBJECT;
-      }
+      // Defer serialization until toData is called.
+      this.oldValueIsSerialized = true; //VALUE_IS_SERIALIZED_OBJECT;
     } else {
       this.oldValueIsSerialized = false; //VALUE_IS_BYTES;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteFetchEntryMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteFetchEntryMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteFetchEntryMessage.java
index b7cc393..b7a4a81 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteFetchEntryMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteFetchEntryMessage.java
@@ -128,9 +128,6 @@ public final class RemoteFetchEntryMessage extends RemoteOperationMessage
     }
     EntrySnapshot val;
       try {
-        if (r.keyRequiresRegionContext()) {
-          ((KeyWithRegionContext)this.key).setRegionContext(r);
-        }
         final KeyInfo keyInfo = r.getKeyInfo(key);
         Region.Entry re = r.getDataView().getEntry(keyInfo, r, true);
         if(re==null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteFetchVersionMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteFetchVersionMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteFetchVersionMessage.java
index 21590f6..124c9b5 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteFetchVersionMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteFetchVersionMessage.java
@@ -122,9 +122,6 @@ public final class RemoteFetchVersionMessage extends RemoteOperationMessage {
     }
     VersionTag tag;
     try {
-      if (r.keyRequiresRegionContext()) {
-        ((KeyWithRegionContext) this.key).setRegionContext(r);
-      }
       RegionEntry re = r.getRegionEntry(key);
       if (re == null) {
         if (logger.isTraceEnabled(LogMarker.DM)) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteGetMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteGetMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteGetMessage.java
index 7e2be1f..05d62d4 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteGetMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteGetMessage.java
@@ -118,9 +118,6 @@ public final class RemoteGetMessage extends RemoteOperationMessageWithDirectRepl
     RawValue valueBytes;
     Object val = null;
       try {
-        if (r.keyRequiresRegionContext()) {
-          ((KeyWithRegionContext)this.key).setRegionContext(r);
-        }
         KeyInfo keyInfo = r.getKeyInfo(key, cbArg);
         val = r.getDataView().getSerializedValue(r, keyInfo, false, this.context, null, false /*for replicate regions*/);
         valueBytes = val instanceof RawValue ? (RawValue)val : new RawValue(val);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteInvalidateMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteInvalidateMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteInvalidateMessage.java
index f975f6f..a4e020e 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteInvalidateMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteInvalidateMessage.java
@@ -188,9 +188,6 @@ public final class RemoteInvalidateMessage extends RemoteDestroyMessage {
        eventSender = getSender();
     }
     final Object key = getKey();
-    if (r.keyRequiresRegionContext()) {
-      ((KeyWithRegionContext)key).setRegionContext(r);
-    }
     @Released final EntryEventImpl event = EntryEventImpl.create(
         r,
         getOperation(),

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutAllMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutAllMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutAllMessage.java
index 045e51c..c0c56c9 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutAllMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutAllMessage.java
@@ -82,7 +82,6 @@ public final class RemotePutAllMessage extends RemoteOperationMessageWithDirectR
 
   protected static final short HAS_BRIDGE_CONTEXT = UNRESERVED_FLAGS_START;
   protected static final short SKIP_CALLBACKS = (HAS_BRIDGE_CONTEXT << 1);
-  protected static final short IS_PUT_DML = (SKIP_CALLBACKS << 1);
 
   private EventID eventId;
   
@@ -92,8 +91,6 @@ public final class RemotePutAllMessage extends RemoteOperationMessageWithDirectR
 
 //  private boolean useOriginRemote;
 
-  private boolean isPutDML;
-  
   public void addEntry(PutAllEntryData entry) {
     this.putAllData[this.putAllDataCount++] = entry;
   }
@@ -190,7 +187,6 @@ public final class RemotePutAllMessage extends RemoteOperationMessageWithDirectR
     this.eventId = event.getEventId();
     this.skipCallbacks = skipCallbacks;
     this.callbackArg = event.getCallbackArgument();
-	this.isPutDML = event.isPutDML();
   }
 
   public RemotePutAllMessage() {
@@ -241,7 +237,6 @@ public final class RemotePutAllMessage extends RemoteOperationMessageWithDirectR
       this.bridgeContext = DataSerializer.readObject(in);
     }
     this.skipCallbacks = (flags & SKIP_CALLBACKS) != 0;
-    this.isPutDML = (flags & IS_PUT_DML) != 0;
     this.putAllDataCount = (int)InternalDataSerializer.readUnsignedVL(in);
     this.putAllData = new PutAllEntryData[putAllDataCount];
     if (this.putAllDataCount > 0) {
@@ -279,10 +274,6 @@ public final class RemotePutAllMessage extends RemoteOperationMessageWithDirectR
       EntryVersionsList versionTags = new EntryVersionsList(putAllDataCount);
 
       boolean hasTags = false;
-      // get the "keyRequiresRegionContext" flag from first element assuming
-      // all key objects to be uniform
-      final boolean requiresRegionContext =
-        (this.putAllData[0].key instanceof KeyWithRegionContext);
       for (int i = 0; i < this.putAllDataCount; i++) {
         if (!hasTags && putAllData[i].versionTag != null) {
           hasTags = true;
@@ -290,7 +281,7 @@ public final class RemotePutAllMessage extends RemoteOperationMessageWithDirectR
         VersionTag<?> tag = putAllData[i].versionTag;
         versionTags.add(tag);
         putAllData[i].versionTag = null;
-        this.putAllData[i].toData(out, requiresRegionContext);
+        this.putAllData[i].toData(out);
         this.putAllData[i].versionTag = tag;
       }
 
@@ -307,7 +298,6 @@ public final class RemotePutAllMessage extends RemoteOperationMessageWithDirectR
     if (this.posDup) flags |= POS_DUP;
     if (this.bridgeContext != null) flags |= HAS_BRIDGE_CONTEXT;
     if (this.skipCallbacks) flags |= SKIP_CALLBACKS;
-    if (this.isPutDML) flags |= IS_PUT_DML;
     return flags;
   }
 
@@ -370,7 +360,6 @@ public final class RemotePutAllMessage extends RemoteOperationMessageWithDirectR
       baseEvent.setContext(this.bridgeContext);
     }
     baseEvent.setPossibleDuplicate(this.posDup);
-	baseEvent.setPutDML(this.isPutDML);
     if (logger.isDebugEnabled()) {
       logger.debug("RemotePutAllMessage.doLocalPutAll: eventSender is {}, baseEvent is {}, msg is {}",
           eventSender, baseEvent, this);
@@ -384,7 +373,7 @@ public final class RemotePutAllMessage extends RemoteOperationMessageWithDirectR
 //        final boolean requiresRegionContext = dr.keyRequiresRegionContext();
         InternalDistributedMember myId = r.getDistributionManager().getDistributionManagerId();
         for (int i = 0; i < putAllDataCount; ++i) {
-          @Released EntryEventImpl ev = PutAllPRMessage.getEventFromEntry(r, myId, eventSender, i, putAllData, false, bridgeContext, posDup, !skipCallbacks, isPutDML);
+          @Released EntryEventImpl ev = PutAllPRMessage.getEventFromEntry(r, myId, eventSender, i, putAllData, false, bridgeContext, posDup, !skipCallbacks);
           try {
           ev.setPutAllOperation(dpao);
           if (logger.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java
index 678927d..34d3585 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemotePutMessage.java
@@ -135,8 +135,7 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
   
   /**
    * Indicates if and when the new value should be deserialized on the
-   * the receiver. Distinguishes between Deltas which need to be eagerly
-   * deserialized (DESERIALIZATION_POLICY_EAGER), a non-byte[] value that was
+   * the receiver. Distinguishes between a non-byte[] value that was
    * serialized (DESERIALIZATION_POLICY_LAZY) and a
    * byte[] array value that didn't need to be serialized
    * (DESERIALIZATION_POLICY_NONE). While this seems like an extra data, it
@@ -252,11 +251,7 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
     event.setOriginRemote(useOriginRemote);
 
     if (event.hasNewValue()) {
-      if (CachedDeserializableFactory.preferObject() || event.hasDelta()) {
-        this.deserializationPolicy = DistributedCacheOperation.DESERIALIZATION_POLICY_EAGER;
-      } else {
-        this.deserializationPolicy = DistributedCacheOperation.DESERIALIZATION_POLICY_LAZY;
-      }
+      this.deserializationPolicy = DistributedCacheOperation.DESERIALIZATION_POLICY_LAZY;
       event.exportNewValue(this);
     }
     else {
@@ -568,13 +563,7 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
       this.oldValueIsSerialized = (in.readByte() == 1);
       setOldValBytes(DataSerializer.readByteArray(in));
     }
-    if (this.deserializationPolicy ==
-        DistributedCacheOperation.DESERIALIZATION_POLICY_EAGER) {
-      setValObj(DataSerializer.readObject(in));
-    }
-    else {
-      setValBytes(DataSerializer.readByteArray(in));
-    }
+    setValBytes(DataSerializer.readByteArray(in));
     if ((flags & HAS_DELTA_BYTES) != 0) {
       this.applyDeltaBytes = true;
       this.deltaBytes = DataSerializer.readByteArray(in);
@@ -681,9 +670,6 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
     if (eventSender == null) {
        eventSender = getSender();
     }
-    if (r.keyRequiresRegionContext()) {
-      ((KeyWithRegionContext)this.key).setRegionContext(r);
-    }
     @Released EntryEventImpl eei = EntryEventImpl.create(
         r,
         getOperation(),
@@ -732,10 +718,6 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
         case DistributedCacheOperation.DESERIALIZATION_POLICY_NONE:
           event.setNewValue(getValBytes());
           break;
-        case DistributedCacheOperation.DESERIALIZATION_POLICY_EAGER:
-          // new value is a Delta
-          event.setNewValue(this.valObj); // sets the delta field
-          break;
         default:
           throw new AssertionError("unknown deserialization policy: "
               + deserializationPolicy);
@@ -1212,12 +1194,8 @@ public final class RemotePutMessage extends RemoteOperationMessageWithDirectRepl
   
   private void setOldValueIsSerialized(boolean isSerialized) {
     if (isSerialized) {
-      if (CachedDeserializableFactory.preferObject()) {
-        this.oldValueIsSerialized = true; //VALUE_IS_OBJECT;
-      } else {
-        // Defer serialization until toData is called.
-        this.oldValueIsSerialized = true; //VALUE_IS_SERIALIZED_OBJECT;
-      }
+      // Defer serialization until toData is called.
+      this.oldValueIsSerialized = true; //VALUE_IS_SERIALIZED_OBJECT;
     } else {
       this.oldValueIsSerialized = false; //VALUE_IS_BYTES;
     }



[2/5] incubator-geode git commit: GEODE-1464: remove sqlf code

Posted by ds...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteRemoveAllMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteRemoveAllMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteRemoveAllMessage.java
index 2d5989b..da60a98 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteRemoveAllMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteRemoveAllMessage.java
@@ -267,10 +267,6 @@ public final class RemoteRemoveAllMessage extends RemoteOperationMessageWithDire
       EntryVersionsList versionTags = new EntryVersionsList(removeAllDataCount);
 
       boolean hasTags = false;
-      // get the "keyRequiresRegionContext" flag from first element assuming
-      // all key objects to be uniform
-      final boolean requiresRegionContext =
-        (this.removeAllData[0].key instanceof KeyWithRegionContext);
       for (int i = 0; i < this.removeAllDataCount; i++) {
         if (!hasTags && removeAllData[i].versionTag != null) {
           hasTags = true;
@@ -278,7 +274,7 @@ public final class RemoteRemoveAllMessage extends RemoteOperationMessageWithDire
         VersionTag<?> tag = removeAllData[i].versionTag;
         versionTags.add(tag);
         removeAllData[i].versionTag = null;
-        this.removeAllData[i].toData(out, requiresRegionContext);
+        this.removeAllData[i].toData(out);
         this.removeAllData[i].versionTag = tag;
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
index bd19104..4f49ccf 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/SearchLoadAndWriteProcessor.java
@@ -1898,9 +1898,6 @@ public class SearchLoadAndWriteProcessor implements MembershipListener {
           setClearCountReference(region);		
 	  try {
 	    boolean initialized = region.isInitialized();
-	    if(region.keyRequiresRegionContext()) {
-	      ((KeyWithRegionContext)this.key).setRegionContext(region);
-	    }
             RegionEntry entry = region.basicGetEntry(this.key);
             if (entry != null) {
               synchronized (entry) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXEntry.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXEntry.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXEntry.java
index 2cc6681..317f4e6 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXEntry.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXEntry.java
@@ -56,17 +56,6 @@ public class TXEntry implements Region.Entry
     this.keyInfo = key;
     this.myTX = tx;
     this.rememberReads = rememberReads;
-    
-    //Assert that these contructors are invoked only 
-    // via factory path. I am not able to make them private
-    // because SqlfabricTxEntry needs extending TxEntry
-    /*if(logger.isDebugEnabled()) {
-      StackTraceElement[] traces =Thread.currentThread().getStackTrace();
-      //The third element should be the factory one
-      StackTraceElement trace = traces[2];
-      Assert.assertTrue(TxEntryFactory.class.isAssignableFrom(trace.getClass()));
-      Assert.assertTrue(trace.getMethodName().equals("createEntry"));
-    }*/
   }
 
   public boolean isLocal() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXEntryState.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXEntryState.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXEntryState.java
index f392f04..c6caefa 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXEntryState.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXEntryState.java
@@ -24,7 +24,6 @@ import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedM
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
 import com.gemstone.gemfire.internal.Version;
-import com.gemstone.gemfire.internal.cache.delta.Delta;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.lang.StringUtils;
@@ -82,10 +81,6 @@ public class TXEntryState implements Releasable
    */
   private int nearSideEventOffset = -1;
 
-  //Asif: In case of Sqlfabric, the pending value may be a SerializableDelta object 
-  //which may be containing base value ( in case of Tx create) along with bunch 
-  //of incremental deltas, so for correct behaviour this field should be accessed only 
-  //by its getter. Do not use it directly  
   private Object pendingValue;
   
   /**
@@ -290,24 +285,12 @@ public class TXEntryState implements Releasable
   }
 
   public Object getOriginalValue() {
-    Object value = this.originalValue;
-    
-    if(value instanceof Delta) {
-      value = ((Delta) value).getResultantValue();
-    }
-    
-    return value;
+    return this.originalValue;
   }
 
   public Object getPendingValue()
   {
-    Object value = this.pendingValue;
-    
-    if(value instanceof Delta) {
-      value = ((Delta) value).getResultantValue();
-    }
-    
-    return value;
+    return this.pendingValue;
   }
   
   public Object getCallbackArgument()
@@ -335,12 +318,7 @@ public class TXEntryState implements Releasable
 
   void setPendingValue(Object pv)
   {
-    if(pv instanceof Delta) {
-      Object toMerge = this.pendingValue;      
-      this.pendingValue = ((Delta)pv).merge(toMerge, this.op == OP_CREATE);
-    }else {
-      this.pendingValue = pv;
-    }
+    this.pendingValue = pv;
   }
   
   void setCallbackArgument(Object callbackArgument)
@@ -2001,9 +1979,6 @@ public class TXEntryState implements Releasable
         valueBytes = (byte[])v;
       }
       else {
-        // this value shouldn't be a Delta
-        Assert.assertTrue(!(v instanceof Delta));
-    
         deserializationPolicy = DistributedCacheOperation.DESERIALIZATION_POLICY_LAZY;
         valueBytes = EntryEventImpl.serialize(v);
       }
@@ -2076,7 +2051,6 @@ public class TXEntryState implements Releasable
   }
   
 
-  // Asif:Add for sql fabric as it has to plug in its own TXEntry object
   private final static TXEntryStateFactory factory = new TXEntryStateFactory() {
 
     public TXEntryState createEntry()

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionLockRequestImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionLockRequestImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionLockRequestImpl.java
index 6d997c7..6327ed7 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionLockRequestImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionLockRequestImpl.java
@@ -104,24 +104,17 @@ public class TXRegionLockRequestImpl
     final GemFireCacheImpl cache = getCache(false);
     try {
       final int size = InternalDataSerializer.readArrayLength(in);
-      boolean read = false;
       if (cache != null && size > 0) {
         this.r = (LocalRegion)cache.getRegion(this.regionPath);
-        if( this.r != null ) {
-          this.entryKeys = readEntryKeySet(this.r.keyRequiresRegionContext(), size, in);
-          read = true;
-        }
-      }
-      if ( !read && size > 0 ) {
-        this.entryKeys = readEntryKeySet(false, size, in);
       }
+      this.entryKeys = readEntryKeySet(size, in);
     } catch (CacheClosedException cce) {
       // don't throw in deserialization
       this.entryKeys = null;
     }
   }
   
-  private final Set<Object> readEntryKeySet(final boolean keyRequiresRegionContext,
+  private final Set<Object> readEntryKeySet(
       final int size, final DataInput in) throws IOException,
       ClassNotFoundException {
 
@@ -133,9 +126,6 @@ public class TXRegionLockRequestImpl
     Object key;
     for (int i = 0; i < size; i++) {
       key = DataSerializer.readObject(in);
-      if (keyRequiresRegionContext) {
-        ((KeyWithRegionContext)key).setRegionContext(this.r);
-      }
       set.add(key);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java
index 19cbe33..479beb2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java
@@ -524,12 +524,7 @@ public class TXRegionState {
     return changes;
   }
   
-  public Map<Object,TXEntryState> getEntriesInTxForSqlFabric() {
-    return Collections.unmodifiableMap(this.entryMods);
-  }
-
   public TXState getTXState() {
-    // TODO Auto-generated method stub
     return txState;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXState.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXState.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXState.java
index 0ce049d..c42f63c 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXState.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXState.java
@@ -1325,13 +1325,10 @@ public class TXState implements TXStateInterface {
    * @param rememberRead true if the value read from committed state
    *   needs to be remembered in tx state for repeatable read.
    * @param createIfAbsent should a transactional entry be created if not present. 
-   *        Used by sql fabric system
    * @return a txEntryState or null if the entry doesn't exist in the transaction and/or committed state. 
    */
   public TXEntryState txReadEntry(KeyInfo keyInfo, LocalRegion localRegion,
       boolean rememberRead, boolean createIfAbsent) {
-    // EntryNotFoundException can be expected in case of sqlfabric and should
-    // not be caught.
     localRegion.cache.getCancelCriterion().checkCancelInProgress(null);
     return txReadEntry(keyInfo, localRegion, rememberRead, null, createIfAbsent);
   }
@@ -1764,7 +1761,7 @@ public class TXState implements TXStateInterface {
 //	        final boolean requiresRegionContext = theRegion.keyRequiresRegionContext();
 	        InternalDistributedMember myId = theRegion.getDistributionManager().getDistributionManagerId();
 	        for (int i = 0; i < putallOp.putAllDataSize; ++i) {
-	          @Released EntryEventImpl ev = PutAllPRMessage.getEventFromEntry(theRegion, myId,myId, i, putallOp.putAllData, false, putallOp.getBaseEvent().getContext(), false, !putallOp.getBaseEvent().isGenerateCallbacks(), false);
+	          @Released EntryEventImpl ev = PutAllPRMessage.getEventFromEntry(theRegion, myId,myId, i, putallOp.putAllData, false, putallOp.getBaseEvent().getContext(), false, !putallOp.getBaseEvent().isGenerateCallbacks());
 	          try {
 	          ev.setPutAllOperation(putallOp);
 	          if (theRegion.basicPut(ev, false, false, null, false)) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateInterface.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateInterface.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateInterface.java
index ffcae4b..865ebd5 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateInterface.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateInterface.java
@@ -146,7 +146,6 @@ public interface TXStateInterface extends Synchronization, InternalDataView {
    * @param rememberRead true if the value read from committed state
    *   needs to be remembered in tx state for repeatable read.
    * @param  createTxEntryIfAbsent should a transactional entry be created if not present. 
-   *        Used by sqlfabric system
    * @return a txEntryState or null if the entry doesn't exist in the transaction and/or committed state. 
    */
   public TXEntryState txReadEntry(KeyInfo entryKey, LocalRegion localRegion, boolean rememberRead

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateAttributesProcessor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateAttributesProcessor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateAttributesProcessor.java
index ceb98f0..9754ab9 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateAttributesProcessor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateAttributesProcessor.java
@@ -369,14 +369,7 @@ public class UpdateAttributesProcessor {
       // set the processor ID to be able to send reply to sender in case of any
       // unexpected exception during deserialization etc.
       ReplyProcessor21.setMessageRPId(this.processorId);
-      try {
-        this.profile = DataSerializer.readObject(in);
-      } catch (DSFIDFactory.SqlfSerializationException ex) {
-        // Ignore SQLFabric serialization errors and reply with nothing.
-        // This can happen even during normal startup of all SQLFabric VMs
-        // when DS connect is complete but SQLFabric boot is still in progress.
-        this.profile = null;
-      }
+      this.profile = DataSerializer.readObject(in);
       this.exchangeProfiles = in.readBoolean();
       this.removeProfile = in.readBoolean();
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateEntryVersionOperation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateEntryVersionOperation.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateEntryVersionOperation.java
index fce4dee..23fb300 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateEntryVersionOperation.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateEntryVersionOperation.java
@@ -94,11 +94,6 @@ public class UpdateEntryVersionOperation extends DistributedCacheOperation {
     @Retained
     protected InternalCacheEvent createEvent(DistributedRegion rgn)
         throws EntryNotFoundException {
-      
-      if (rgn.keyRequiresRegionContext()) {
-        ((KeyWithRegionContext)this.key).setRegionContext(rgn);
-      }
-      
       @Retained EntryEventImpl ev = EntryEventImpl.create(rgn, getOperation(), this.key,
          null /* newValue */, this.callbackArg /*callbackArg*/, true /* originRemote*/ , getSender(), false /*generateCallbacks*/);
       ev.setEventId(this.eventId);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateOperation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateOperation.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateOperation.java
index 730d6d7..e60cda3 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateOperation.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/UpdateOperation.java
@@ -100,11 +100,7 @@ public class UpdateOperation extends AbstractUpdateOperation
     m.event = ev;
     m.eventId = ev.getEventId();
     m.key = ev.getKey();
-    if (CachedDeserializableFactory.preferObject() || ev.hasDelta()) {
-      m.deserializationPolicy = DESERIALIZATION_POLICY_EAGER;
-    } else {
-      m.deserializationPolicy = DESERIALIZATION_POLICY_LAZY;
-    }
+    m.deserializationPolicy = DESERIALIZATION_POLICY_LAZY;
     ev.exportNewValue(m);
   }
 
@@ -297,8 +293,7 @@ public class UpdateOperation extends AbstractUpdateOperation
      */
     static void setNewValueInEvent(byte[] newValue, Object newValueObj,
         EntryEventImpl event, byte deserializationPolicy) {
-      if (newValue == null
-          && deserializationPolicy != DESERIALIZATION_POLICY_EAGER) {
+      if (newValue == null) {
         // in an UpdateMessage this results from a create(key, null) call,
         // set local invalid flag in event if this is a normal region. Otherwise
         // it should be a distributed invalid.
@@ -317,9 +312,6 @@ public class UpdateOperation extends AbstractUpdateOperation
         case DESERIALIZATION_POLICY_NONE:
           event.setNewValue(newValue);
           break;
-        case DESERIALIZATION_POLICY_EAGER:
-          event.setNewValue(newValueObj);
-          break;
         default:
           throw new InternalGemFireError(LocalizedStrings
               .UpdateOperation_UNKNOWN_DESERIALIZATION_POLICY_0
@@ -332,10 +324,6 @@ public class UpdateOperation extends AbstractUpdateOperation
     {
       Object argNewValue = null;
       final boolean originRemote = true, generateCallbacks = true;
-
-      if (rgn.keyRequiresRegionContext()) {
-        ((KeyWithRegionContext)this.key).setRegionContext(rgn);
-      }
       @Retained EntryEventImpl result = EntryEventImpl.create(rgn, getOperation(), this.key,
           argNewValue, // oldValue,
           this.callbackArg, originRemote, getSender(), generateCallbacks);
@@ -413,13 +401,7 @@ public class UpdateOperation extends AbstractUpdateOperation
         this.deltaBytes = DataSerializer.readByteArray(in);
       }
       else {
-        if (this.deserializationPolicy
-            == DistributedCacheOperation.DESERIALIZATION_POLICY_EAGER) {
-          this.newValueObj = DataSerializer.readObject(in);
-        }
-        else {
-          this.newValue = DataSerializer.readByteArray(in);
-        }
+        this.newValue = DataSerializer.readByteArray(in);
         if ((extraFlags & HAS_DELTA_WITH_FULL_VALUE) != 0) {
           this.deltaBytes = DataSerializer.readByteArray(in);
         }
@@ -500,13 +482,7 @@ public class UpdateOperation extends AbstractUpdateOperation
       byte[] valueBytes = null;
       Object valueObj = null;
       if (this.newValueObj != null) {
-        if (this.deserializationPolicy ==
-          DistributedCacheOperation.DESERIALIZATION_POLICY_EAGER) {
-          valueObj = this.newValueObj;
-        }
-        else {
-          valueBytes = EntryEventImpl.serialize(this.newValueObj);
-        }
+        valueBytes = EntryEventImpl.serialize(this.newValueObj);
       }
       else {
         valueBytes = this.newValue;
@@ -576,10 +552,6 @@ public class UpdateOperation extends AbstractUpdateOperation
       // boolean localLoad = false, netLoad = false, netSearch = false,
       // distributed = true;
       final boolean originRemote = true, generateCallbacks = true;
-
-      if (rgn.keyRequiresRegionContext()) {
-        ((KeyWithRegionContext)this.key).setRegionContext(rgn);
-      }
       @Retained EntryEventImpl ev = EntryEventImpl.create(rgn, getOperation(), this.key,
           argNewValue, this.callbackArg, originRemote, getSender(),
           generateCallbacks);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ValidatingDiskRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ValidatingDiskRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ValidatingDiskRegion.java
index 32abd68..edf3316 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ValidatingDiskRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/ValidatingDiskRegion.java
@@ -173,9 +173,6 @@ public class ValidatingDiskRegion extends DiskRegion implements DiskRecoveryStor
     public void handleValueOverflow(RegionEntryContext context) {throw new IllegalStateException("should never be called");}
     
     @Override
-    public void afterValueOverflow(RegionEntryContext context) {throw new IllegalStateException();}
-    
-    @Override
     public Object prepareValueForCache(RegionEntryContext r, Object val, boolean isEntryUpdate) {
       throw new IllegalStateException("Should never be called");
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/WrappedCallbackArgument.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/WrappedCallbackArgument.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/WrappedCallbackArgument.java
index 7d5b255..93fc533 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/WrappedCallbackArgument.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/WrappedCallbackArgument.java
@@ -40,12 +40,6 @@ public abstract class WrappedCallbackArgument {
    */
   private Object _originalCallbackArg;
   
-  /** If the GatewayEvent is in a Sql Fabric started Hub, in which case
-   * the original callback argument is not serialized
-   * 
-   */
-   private boolean serializeCallbackArg = true; 
-
   /**
    * No arg constructor for DataSerializable.
    */
@@ -57,20 +51,6 @@ public abstract class WrappedCallbackArgument {
    *
    * @param originalCallbackArg The original callback argument set by the
    * caller or null if there was not callback arg
-   * @param serializeCBArg  boolean indicating if the event is created by a 
-   * sql fabric system
-   */
-  public WrappedCallbackArgument(Object originalCallbackArg, boolean serializeCBArg) {
-    this._originalCallbackArg = originalCallbackArg;
-    this.serializeCallbackArg = serializeCBArg;
-  }
- 
-  
-  /**
-   * Constructor.
-   *
-   * @param originalCallbackArg The original callback argument set by the
-   * caller or null if there was not callback arg
    */
   public WrappedCallbackArgument(Object originalCallbackArg) {
     this._originalCallbackArg = originalCallbackArg;    
@@ -86,11 +66,7 @@ public abstract class WrappedCallbackArgument {
   }
 
   public void toData(DataOutput out) throws IOException {
-    if(this.serializeCallbackArg) {
-      DataSerializer.writeObject(this._originalCallbackArg, out);
-    }else {
-      DataSerializer.writeObject(null, out);      
-    }
+    DataSerializer.writeObject(this._originalCallbackArg, out);
   }
 
   public void fromData(DataInput in) throws IOException, ClassNotFoundException {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/delta/Delta.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/delta/Delta.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/delta/Delta.java
deleted file mode 100644
index 27d02c4..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/delta/Delta.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package com.gemstone.gemfire.internal.cache.delta;
-
-import com.gemstone.gemfire.cache.EntryEvent;
-
-/**
- * Represents changes to apply to a region entry instead of a new value.
- * A Delta is passed as the new value in a put operation on a Region
- * and knows how to apply itself to an old value.
- *
- * Internal Note: When an update message carries a Delta as a payload,
- * it makes sure it gets deserialized before being put into the region.
- *
- * @since GemFire 5.5
- * @see com.gemstone.gemfire.internal.cache.UpdateOperation
- */
-public interface Delta {
-
-  /**
-   * Apply delta to the old value from the provided EntryEvent.
-   * If the delta cannot be applied for any reason then an (unchecked)
-   * exception should be thrown. If the put is being applied in a
-   * distributed-ack scope, then the exception will be propagated back
-   * to the originating put call, but will not necessarily cause puts
-   * in other servers to fail.
-   *
-   * @param putEvent the EntryEvent for the put operation, from which
-   * the old value can be obtained (as well as other information such
-   * as the key and region being operated on)
-   *
-   * @return the new value to be put into the region
-   */
-  Object apply(EntryEvent<?, ?> putEvent);
-
-  Object merge(Object toMerge, boolean isCreate);
-
-  Object merge(Object toMerge);
-
-  Object getResultantValue();
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/AbstractExecution.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/AbstractExecution.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/AbstractExecution.java
index 7b7c34d..fefc1c1 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/AbstractExecution.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/AbstractExecution.java
@@ -159,7 +159,6 @@ public abstract class AbstractExecution implements InternalExecution {
   }
   
   protected AbstractExecution() {
-    this.hasRoutingObjects = false;
   }
 
   protected AbstractExecution(AbstractExecution ae) {
@@ -173,7 +172,6 @@ public abstract class AbstractExecution implements InternalExecution {
       this.memberMappedArg = ae.memberMappedArg;
     }
     this.isMemberMappedArgument = ae.isMemberMappedArgument;
-    this.hasRoutingObjects = ae.hasRoutingObjects;
     this.isClientServerMode = ae.isClientServerMode;
     if (ae.proxyCache != null) {
       this.proxyCache = ae.proxyCache;
@@ -211,26 +209,10 @@ public abstract class AbstractExecution implements InternalExecution {
     return this.rc;
   }
 
-  public AbstractExecution withRoutingObjects(Set<Object> routingObjects) {
-    if (routingObjects == null) {
-      throw new FunctionException(
-          LocalizedStrings.FunctionService_ROUTING_OBJECTS_SET_IS_NULL
-              .toLocalizedString());
-    }
-    this.filter.clear();
-    this.filter.addAll(routingObjects);
-    this.hasRoutingObjects = true;
-    return this;
-  }
-
   public Set getFilter() {
     return this.filter;
   }
 
-  public boolean hasRoutingObjects() {
-    return this.hasRoutingObjects;
-  }
-
   public AbstractExecution setIsReExecute() {
     this.isReExecute = true;
     if (this.executionNodesListener != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/FunctionStreamingResultCollector.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/FunctionStreamingResultCollector.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/FunctionStreamingResultCollector.java
index 3874ad9..7ed908e 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/FunctionStreamingResultCollector.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/FunctionStreamingResultCollector.java
@@ -96,8 +96,7 @@ public class FunctionStreamingResultCollector extends ReplyProcessor21 implement
     this.execution = execution;
     this.fites = Collections.synchronizedList(new ArrayList<FunctionInvocationTargetException>());
     // add a reference to self inside the ResultCollector, if required, to avoid
-    // this ReplyProcessor21 from being GCed; currently is of use for SQLFabric
-    // result collectors to properly implement streaming
+    // this ReplyProcessor21 from being GCed
     if (rc instanceof LocalResultCollector<?, ?>) {
       ((LocalResultCollector<?, ?>)rc).setProcessor(this);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalExecution.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalExecution.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalExecution.java
index 5ffd72d..81944ef 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalExecution.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalExecution.java
@@ -25,7 +25,7 @@ import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.execute.ResultCollector;
 
 /**
- * Internal interface for SQLFabric. It has internal methods specific for SQLFabric
+ * Internal interface that adds some internal methods to the Execution interface.
  * 
  * @since GemFire 5.8LA
  * 
@@ -36,31 +36,6 @@ public interface InternalExecution extends Execution {
       MemberMappedArgument argument); 
 
   /**
-   * Specifies a data filter of routing objects for selecting the GemFire
-   * members to execute the function that are not GemFire keys rather routing
-   * objects as determined by resolver. Currently used by SQL fabric for passing
-   * routing objects obtained from the custom resolvers.
-   * <p>
-   * If the set is empty the function is executed on all members that have the
-   * {@linkplain FunctionService#onRegion(com.gemstone.gemfire.cache.Region)
-   * region defined}.
-   * </p>
-   * 
-   * @param routingObjects
-   *          Set defining the routing objects to be used for executing the
-   *          function.
-   * 
-   * @return an Execution with the routing objects
-   * 
-   * @throws IllegalArgumentException
-   *           if the set of routing objects passed is null.
-   * @throws UnsupportedOperationException
-   *           if not called after
-   *           {@link FunctionService#onRegion(com.gemstone.gemfire.cache.Region)}
-   */
-  public InternalExecution withRoutingObjects(Set<Object> routingObjects);
-  
-  /**
    * Specifies a  filter of bucketIDs for selecting the GemFire
    * members to execute the function on.
    * <p>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalFunctionService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalFunctionService.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalFunctionService.java
index 54e03c7..f0ed757 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalFunctionService.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalFunctionService.java
@@ -31,7 +31,7 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
 /**
  * 
- * Provides internal methods for sqlFabric product
+ * Provides internal methods for tests
  * 
  * 
  * @since GemFire 6.1

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalRegionFunctionContext.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalRegionFunctionContext.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalRegionFunctionContext.java
index 2234925..90ad79c 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalRegionFunctionContext.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/InternalRegionFunctionContext.java
@@ -30,11 +30,6 @@ import com.gemstone.gemfire.internal.cache.LocalDataSet;
 /**
  * Internal interface used to provide for some essential functionality for
  * {@link RegionFunctionContext} invoked by {@link PartitionRegionHelper}.
- * SQLFabric provides its own implementation when using function messages
- * instead of {@link Function}s so {@link PartitionRegionHelper} should not
- * depend on casting to {@link RegionFunctionContextImpl} directly rather should
- * use this interface.
- * 
  */
 public interface InternalRegionFunctionContext extends RegionFunctionContext {
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/MemberFunctionExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/MemberFunctionExecutor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/MemberFunctionExecutor.java
index ac628a0..3912245 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/MemberFunctionExecutor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/MemberFunctionExecutor.java
@@ -239,13 +239,6 @@ public class MemberFunctionExecutor extends AbstractExecution {
             .toLocalizedString("bucket as filter"));
   }
 
-  @Override
-  public AbstractExecution withRoutingObjects(Set<Object> routingObjects) {
-    throw new FunctionException(
-        LocalizedStrings.ExecuteFunction_CANNOT_SPECIFY_0_FOR_DATA_INDEPENDENT_FUNCTIONS
-            .toLocalizedString("routing objects"));
-  }
-
   public InternalExecution withMemberMappedArgument(
       MemberMappedArgument argument) {
     if(argument == null){

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/MultiRegionFunctionExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/MultiRegionFunctionExecutor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/MultiRegionFunctionExecutor.java
index 1929169..a9f933a 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/MultiRegionFunctionExecutor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/MultiRegionFunctionExecutor.java
@@ -182,13 +182,6 @@ public class MultiRegionFunctionExecutor extends AbstractExecution {
   }
 
   @Override
-  public AbstractExecution withRoutingObjects(Set<Object> routingObjects) {
-    throw new FunctionException(
-        LocalizedStrings.ExecuteFunction_CANNOT_SPECIFY_0_FOR_ONREGIONS_FUNCTION
-            .toLocalizedString("routing objects"));
-  }
-
-  @Override
   protected ResultCollector executeFunction(Function function) {
     if (function.hasResult()) {
       ResultCollector rc = this.rc;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/RegionFunctionContextImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/RegionFunctionContextImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/RegionFunctionContextImpl.java
index 15d39c9..302b24e 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/RegionFunctionContextImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/RegionFunctionContextImpl.java
@@ -26,9 +26,7 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.execute.Execution;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.execute.ResultSender;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.LocalDataSet;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
 
 /**
  * Context available to data dependent functions. When function is executed
@@ -67,17 +65,6 @@ public class RegionFunctionContextImpl extends FunctionContextImpl implements
     this.localBucketSet = localBucketSet;
     this.isPossibleDuplicate = isPossibleDuplicate;
     setFunctionContexts();
-    // set the region context for keys if required
-    if (routingObjects != null) {
-      final LocalRegion r = (LocalRegion)this.dataSet;
-      if (r.keyRequiresRegionContext()) {
-        for (Object key : routingObjects) {
-          if (key instanceof KeyWithRegionContext) {
-            ((KeyWithRegionContext)key).setRegionContext(r);
-          }
-        }
-      }
-    }
   }
 
   private void setFunctionContexts() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/ServerFunctionExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/ServerFunctionExecutor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/ServerFunctionExecutor.java
index 9c06bf6..d821b32 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/ServerFunctionExecutor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/ServerFunctionExecutor.java
@@ -266,13 +266,6 @@ public class ServerFunctionExecutor extends AbstractExecution {
             .toLocalizedString("buckets as filter"));
   }
 
-  @Override
-  public AbstractExecution withRoutingObjects(Set<Object> routingObjects) {
-    throw new FunctionException(
-        LocalizedStrings.ExecuteFunction_CANNOT_SPECIFY_0_FOR_DATA_INDEPENDENT_FUNCTIONS
-            .toLocalizedString("routing objects"));
-  }
-
   public Execution withArgs(Object args) {
     if (args == null) {
       throw new FunctionException(

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/ContainsKeyValueMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/ContainsKeyValueMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/ContainsKeyValueMessage.java
index f3f534a..ffdfde0 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/ContainsKeyValueMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/ContainsKeyValueMessage.java
@@ -38,7 +38,6 @@ import com.gemstone.gemfire.distributed.internal.ReplySender;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
 import com.gemstone.gemfire.internal.cache.PrimaryBucketException;
@@ -125,9 +124,6 @@ public final class ContainsKeyValueMessage extends PartitionMessageWithDirectRep
     final boolean replyVal;
     if (ds != null) {
       try {
-        if (r.keyRequiresRegionContext()) {
-          ((KeyWithRegionContext)this.key).setRegionContext(r);
-        }
         if (this.valueCheck) {
           replyVal = ds.containsValueForKeyLocally(this.bucketId, this.key);
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/DestroyMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/DestroyMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/DestroyMessage.java
index 2700c61..3bd32cc 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/DestroyMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/DestroyMessage.java
@@ -48,7 +48,6 @@ import com.gemstone.gemfire.internal.cache.EnumListenerEvent;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.FilterRoutingInfo;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionHelper;
@@ -253,9 +252,6 @@ public class DestroyMessage extends PartitionMessageWithDirectReply {
     }
     @Released EntryEventImpl event = null;
     try {
-    if (r.keyRequiresRegionContext()) {
-      ((KeyWithRegionContext)this.key).setRegionContext(r);
-    }
     if (this.bridgeContext != null) {
       event = EntryEventImpl.create(r, getOperation(), this.key, null/*newValue*/,
           getCallbackArg(), false/*originRemote*/, eventSender, 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchBulkEntriesMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchBulkEntriesMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchBulkEntriesMessage.java
index 8a6563b..efd0bea 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchBulkEntriesMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchBulkEntriesMessage.java
@@ -52,7 +52,6 @@ import com.gemstone.gemfire.internal.cache.BucketRegion;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
 import com.gemstone.gemfire.internal.cache.InitialImageOperation;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
 import com.gemstone.gemfire.internal.cache.VersionTagHolder;
@@ -523,8 +522,6 @@ public final class FetchBulkEntriesMessage extends PartitionMessage
         try {
           ByteArrayInputStream byteStream = new ByteArrayInputStream(msg.chunk);
           DataInputStream in = new DataInputStream(byteStream);
-          final boolean requiresRegionContext = this.pr
-              .keyRequiresRegionContext();
           Object key;
           int currentId;
 
@@ -538,9 +535,6 @@ public final class FetchBulkEntriesMessage extends PartitionMessage
               deserializingKey = true;
               key = DataSerializer.readObject(in);
               if (key != null) {
-                if (requiresRegionContext) {
-                  ((KeyWithRegionContext) key).setRegionContext(this.pr);
-                }
                 deserializingKey = false;
                 Object value = DataSerializer.readObject(in);
                 VersionTag versionTag = DataSerializer.readObject(in);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessage.java
index d7a3a5f..a18837a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntriesMessage.java
@@ -51,7 +51,6 @@ import com.gemstone.gemfire.internal.cache.BucketRegion;
 import com.gemstone.gemfire.internal.cache.CachedDeserializable;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
 import com.gemstone.gemfire.internal.cache.InitialImageOperation;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
@@ -542,17 +541,12 @@ public final class FetchEntriesMessage extends PartitionMessage
         try {
           ByteArrayInputStream byteStream = new ByteArrayInputStream(msg.chunk);
           DataInputStream in = new DataInputStream(byteStream);
-          final boolean requiresRegionContext = this.pr
-              .keyRequiresRegionContext();
           Object key;
           
           while (in.available() > 0) {
             deserializingKey = true;
             key = DataSerializer.readObject(in);
             if (key != null) {
-              if (requiresRegionContext) {
-                ((KeyWithRegionContext)key).setRegionContext(this.pr);
-              }
               deserializingKey = false;
               Object value = DataSerializer.readObject(in);
               VersionTag versionTag = DataSerializer.readObject(in);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntryMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntryMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntryMessage.java
index fafb546..82f9efb 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntryMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchEntryMessage.java
@@ -46,7 +46,6 @@ import com.gemstone.gemfire.internal.cache.DataLocationException;
 import com.gemstone.gemfire.internal.cache.EntrySnapshot;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
 import com.gemstone.gemfire.internal.cache.KeyInfo;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
 import com.gemstone.gemfire.internal.cache.PrimaryBucketException;
@@ -146,9 +145,6 @@ public final class FetchEntryMessage extends PartitionMessage
     EntrySnapshot val;
     if (ds != null) {
       try {
-        if (r.keyRequiresRegionContext()) {
-          ((KeyWithRegionContext)this.key).setRegionContext(r);
-        }
         KeyInfo keyInfo = r.getKeyInfo(key);
         val = (EntrySnapshot)r.getDataView().getEntryOnRemote(keyInfo, r, true);
         r.getPrStats().endPartitionMessagesProcessing(startTime); 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchKeysMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchKeysMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchKeysMessage.java
index 3faf1da..6714271 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchKeysMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/FetchKeysMessage.java
@@ -44,7 +44,6 @@ import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
 import com.gemstone.gemfire.internal.cache.InitialImageOperation;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
 import com.gemstone.gemfire.internal.cache.tier.InterestType;
@@ -495,14 +494,9 @@ public final class FetchKeysMessage extends PartitionMessage
         try {
           ByteArrayInputStream byteStream = new ByteArrayInputStream(msg.chunk);
           DataInputStream in = new DataInputStream(byteStream);
-          final boolean requiresRegionContext = this.pr
-              .keyRequiresRegionContext();
           while (in.available() > 0) {
             Object key = DataSerializer.readObject(in);
             if (key != null) {
-              if (requiresRegionContext) {
-                ((KeyWithRegionContext)key).setRegionContext(this.pr);
-              }
               synchronized(returnValue) {
                 returnValue.add(key);
               }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java
index e903def..a0d4f63 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java
@@ -29,6 +29,18 @@ import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.*;
 import com.gemstone.gemfire.internal.cache.BucketRegion.RawValue;
+import com.gemstone.gemfire.internal.cache.CachedDeserializableFactory;
+import com.gemstone.gemfire.internal.cache.DataLocationException;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl;
+import com.gemstone.gemfire.internal.cache.ForceReattemptException;
+import com.gemstone.gemfire.internal.cache.KeyInfo;
+import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
+import com.gemstone.gemfire.internal.cache.PrimaryBucketException;
+import com.gemstone.gemfire.internal.cache.TXManagerImpl;
+import com.gemstone.gemfire.internal.cache.TXStateProxy;
+import com.gemstone.gemfire.internal.cache.Token;
+import com.gemstone.gemfire.internal.cache.VersionTagHolder;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
@@ -159,9 +171,6 @@ public final class GetMessage extends PartitionMessageWithDirectReply
     if (ds != null) {
       VersionTagHolder event = new VersionTagHolder();
       try {
-        if (r.keyRequiresRegionContext()) {
-          ((KeyWithRegionContext)this.key).setRegionContext(r);
-        }
         KeyInfo keyInfo = r.getKeyInfo(key, cbArg);
         boolean lockEntry = forceUseOfPRExecutor || isDirectAck();
         
@@ -325,7 +334,6 @@ public final class GetMessage extends PartitionMessageWithDirectReply
     // static values for valueType
     static final byte VALUE_IS_SERIALIZED_OBJECT = 0;
     static final byte VALUE_IS_BYTES = 1;
-    /** came from partial SQLF merge and reconciling with it but not used yet */
     //static final byte VALUE_IS_OBJECT = 2;
     static final byte VALUE_IS_INVALID = 3;
     static final byte VALUE_IS_TOMBSTONE = 4;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/InvalidateMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/InvalidateMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/InvalidateMessage.java
index 6f86acd..bbb9753 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/InvalidateMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/InvalidateMessage.java
@@ -45,7 +45,6 @@ import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EnumListenerEvent;
 import com.gemstone.gemfire.internal.cache.FilterRoutingInfo;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionHelper;
@@ -177,9 +176,6 @@ public final class InvalidateMessage extends DestroyMessage {
        eventSender = getSender();
     }
     final Object key = getKey();
-    if (r.keyRequiresRegionContext()) {
-      ((KeyWithRegionContext)key).setRegionContext(r);
-    }
     @Released final EntryEventImpl event = EntryEventImpl.create(
         r,
         getOperation(),

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PREntriesIterator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PREntriesIterator.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PREntriesIterator.java
index ec2a8db..94a3670 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PREntriesIterator.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PREntriesIterator.java
@@ -22,11 +22,9 @@ import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 
 
 /**
- * This interface is implemented by the iterators
- * GemFireContainer.PRLocalEntriesIterator,
- * PartitionedRegion.PRLocalBucketSetEntriesIterator and
- * PartitionedRegion.KeysSetIterator used by SqlFabric to obtain information of
- * the bucket ID from which the current local entry is being fetched from.
+ * This interface provides the
+ * bucket ID from which the current local entry is being fetched from
+ * and the PartitionedRegion being iterated.
  * 
  */
 public interface PREntriesIterator<T> extends Iterator<T>{

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRUpdateEntryVersionMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRUpdateEntryVersionMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRUpdateEntryVersionMessage.java
index e1766dc..726246b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRUpdateEntryVersionMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PRUpdateEntryVersionMessage.java
@@ -42,7 +42,6 @@ import com.gemstone.gemfire.internal.cache.DataLocationException;
 import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionHelper;
@@ -126,10 +125,6 @@ public class PRUpdateEntryVersionMessage extends
   protected boolean operateOnPartitionedRegion(DistributionManager dm,
       PartitionedRegion pr, long startTime) throws CacheException,
       QueryException, DataLocationException, InterruptedException, IOException {
-    if (pr.keyRequiresRegionContext()) {
-      ((KeyWithRegionContext) key).setRegionContext(pr);
-    }
-
     // release not needed because disallowOffHeapValues called
     final EntryEventImpl event = EntryEventImpl.create(pr, getOperation(),
         getKey(), null, /* newValue */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionMessage.java
index c2ab27e..1b83ee3 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionMessage.java
@@ -468,10 +468,6 @@ public abstract class PartitionMessage extends DistributionMessage implements
     this.processorId = processor == null? 0 : processor.getProcessorId();
     this.notificationOnly = true;
         
-    //Set sqlfAsyncListenerRecepients = r.getRegionAdvisor().adviseSqlfAsyncEventListenerHub();
-    //sqlfAsyncListenerRecepients.retainAll(adjunctRecipients);
-    //Now remove those adjunct recepients which are present in SqlfAsyncListenerRecepients
-    //adjunctRecipients.removeAll(sqlfAsyncListenerRecepients);
     this.setFilterInfo(filterRoutingInfo);
     Set failures1= null;
     if(!adjunctRecipients.isEmpty()) {
@@ -482,20 +478,6 @@ public abstract class PartitionMessage extends DistributionMessage implements
       setRecipients(adjunctRecipients);
       failures1 = r.getDistributionManager().putOutgoing(this);
     }
-    /*
-    //Now distribute message with old value to Sqlf Hub nodes
-    if(!sqlfAsyncListenerRecepients.isEmpty()) {
-      //System.out.println("Asif1: sqlf hub  recepients ="+sqlfHubRecepients);
-      resetRecipients();
-      setRecipients(sqlfAsyncListenerRecepients);
-      event.applyDelta(true);
-      Set failures2 = r.getDistributionManager().putOutgoing(this);
-      if(failures1 == null) {
-        failures1 = failures2;
-      }else if(failures2 != null) {
-        failures1.addAll(failures2);
-      }
-    }*/
     
     return failures1;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java
index 7bbe5ce..f7c63f9 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutAllPRMessage.java
@@ -59,7 +59,6 @@ import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EnumListenerEvent;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
@@ -101,8 +100,6 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
 
   protected static final short HAS_BRIDGE_CONTEXT = UNRESERVED_FLAGS_START;
   protected static final short SKIP_CALLBACKS = (HAS_BRIDGE_CONTEXT << 1);
-  //using the left most bit for IS_PUT_DML, the last available bit
-  protected static final short IS_PUT_DML = (short) (SKIP_CALLBACKS << 1);
 
   private transient InternalDistributedSystem internalDs;
 
@@ -117,7 +114,6 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
   
   transient VersionedObjectList versions = null;
 
-  private boolean isPutDML;
   /**
    * Empty constructor to satisfy {@link DataSerializer}requirements
    */
@@ -125,7 +121,7 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
   }
 
   public PutAllPRMessage(int bucketId, int size, boolean notificationOnly,
-      boolean posDup, boolean skipCallbacks, Object callbackArg, boolean isPutDML) {
+      boolean posDup, boolean skipCallbacks, Object callbackArg) {
     this.bucketId = Integer.valueOf(bucketId);
     putAllPRData = new PutAllEntryData[size];
     this.notificationOnly = notificationOnly;
@@ -133,7 +129,6 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
     this.skipCallbacks = skipCallbacks;
     this.callbackArg = callbackArg;
     initTxMemberId();
-    this.isPutDML = isPutDML;
   }
 
   public void addEntry(PutAllEntryData entry) {
@@ -270,10 +265,6 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
       EntryVersionsList versionTags = new EntryVersionsList(putAllPRDataSize);
 
       boolean hasTags = false;
-      // get the "keyRequiresRegionContext" flag from first element assuming
-      // all key objects to be uniform
-      final boolean requiresRegionContext =
-        (this.putAllPRData[0].getKey() instanceof KeyWithRegionContext);
       for (int i = 0; i < this.putAllPRDataSize; i++) {
         // If sender's version is >= 7.0.1 then we can send versions list.
         if (!hasTags && putAllPRData[i].versionTag != null) {
@@ -283,7 +274,7 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
         VersionTag<?> tag = putAllPRData[i].versionTag;
         versionTags.add(tag);
         putAllPRData[i].versionTag = null;
-        putAllPRData[i].toData(out, requiresRegionContext);
+        putAllPRData[i].toData(out);
         putAllPRData[i].versionTag = tag;
         // PutAllEntryData's toData did not serialize eventID to save
         // performance for DR, but in PR,
@@ -302,7 +293,6 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
     s = super.computeCompressedShort(s);
     if (this.bridgeContext != null) s |= HAS_BRIDGE_CONTEXT;
     if (this.skipCallbacks) s |= SKIP_CALLBACKS;
-    if (this.isPutDML) s |= IS_PUT_DML;
     return s;
   }
 
@@ -311,7 +301,6 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
       ClassNotFoundException {
     super.setBooleans(s, in);
     this.skipCallbacks = ((s & SKIP_CALLBACKS) != 0);
-    this.isPutDML = ((s & IS_PUT_DML) != 0);
   }
 
   @Override
@@ -436,12 +425,8 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
     // Fix the updateMsg misorder issue
     // Lock the keys when doing postPutAll
     Object keys[] = new Object[putAllPRDataSize];
-    final boolean keyRequiresRegionContext = r.keyRequiresRegionContext();
     for (int i = 0; i < putAllPRDataSize; ++i) {
       keys[i] = putAllPRData[i].getKey();
-      if (keyRequiresRegionContext) {
-        ((KeyWithRegionContext)keys[i]).setRegionContext(r);
-      }
     }
 
     if (!notificationOnly) {
@@ -482,7 +467,7 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
            * in this request, because these request will be blocked by foundKey
            */
           for (int i=0; i<putAllPRDataSize; i++) {
-            @Released EntryEventImpl ev = getEventFromEntry(r, myId, eventSender, i,putAllPRData,notificationOnly,bridgeContext,posDup,skipCallbacks, this.isPutDML);
+            @Released EntryEventImpl ev = getEventFromEntry(r, myId, eventSender, i,putAllPRData,notificationOnly,bridgeContext,posDup,skipCallbacks);
             try {
             key = ev.getKey();
 
@@ -558,7 +543,7 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
       }
     } else {
       for (int i=0; i<putAllPRDataSize; i++) {
-        EntryEventImpl ev = getEventFromEntry(r, myId, eventSender, i,putAllPRData,notificationOnly,bridgeContext,posDup,skipCallbacks, this.isPutDML);
+        EntryEventImpl ev = getEventFromEntry(r, myId, eventSender, i,putAllPRData,notificationOnly,bridgeContext,posDup,skipCallbacks);
         try {
         ev.setOriginRemote(true);
         if (this.callbackArg != null) {
@@ -594,7 +579,7 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
       InternalDistributedMember myId, InternalDistributedMember eventSender,
       int idx, DistributedPutAllOperation.PutAllEntryData[] data,
       boolean notificationOnly, ClientProxyMembershipID bridgeContext,
-      boolean posDup, boolean skipCallbacks, boolean isPutDML) {
+      boolean posDup, boolean skipCallbacks) {
     PutAllEntryData prd = data[idx];
     //EntryEventImpl ev = EntryEventImpl.create(r, 
        // prd.getOp(),
@@ -633,7 +618,6 @@ public final class PutAllPRMessage extends PartitionMessageWithDirectReply
     } else {
       ev.setTailKey(prd.getTailKey());
     }
-    ev.setPutDML(isPutDML);
     evReturned = true;
     return ev;
     } finally {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
index db137c6..e63d288 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PutMessage.java
@@ -56,7 +56,6 @@ import com.gemstone.gemfire.internal.cache.EnumListenerEvent;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.FilterRoutingInfo;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
 import com.gemstone.gemfire.internal.cache.PrimaryBucketException;
@@ -119,8 +118,7 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
   
   /**
    * Indicates if and when the new value should be deserialized on the
-   * the receiver. Distinguishes between Deltas which need to be eagerly
-   * deserialized (DESERIALIZATION_POLICY_EAGER), a non-byte[] value that was
+   * the receiver. Distinguishes between a non-byte[] value that was
    * serialized (DESERIALIZATION_POLICY_LAZY) and a
    * byte[] array value that didn't need to be serialized
    * (DESERIALIZATION_POLICY_NONE). While this seems like an extra data, it
@@ -182,15 +180,11 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
 
   private VersionTag versionTag;
 
-  private transient boolean isPutDML;
-  
   // additional bitmask flags used for serialization/deserialization
 
   protected static final short CACHE_WRITE = UNRESERVED_FLAGS_START;
   protected static final short HAS_EXPECTED_OLD_VAL = (CACHE_WRITE << 1);
   protected static final short HAS_VERSION_TAG = (HAS_EXPECTED_OLD_VAL << 1);
-  //using the left most bit for IS_PUT_DML, the last available bit
-  protected static final short IS_PUT_DML = (short) (HAS_VERSION_TAG << 1);
 
   // extraFlags
   protected static final int HAS_BRIDGE_CONTEXT =
@@ -349,11 +343,7 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
     this.expectedOldValue = expectedOldValue;
     this.key = event.getKey();
     if (event.hasNewValue()) {
-      if (CachedDeserializableFactory.preferObject() || event.hasDelta()) {
-        this.deserializationPolicy = DistributedCacheOperation.DESERIALIZATION_POLICY_EAGER;
-      } else {
-        this.deserializationPolicy = DistributedCacheOperation.DESERIALIZATION_POLICY_LAZY;
-      }
+      this.deserializationPolicy = DistributedCacheOperation.DESERIALIZATION_POLICY_LAZY;
       event.exportNewValue(this);
     }
     else {
@@ -627,14 +617,7 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
       this.deltaBytes = DataSerializer.readByteArray(in);
     }
     else {
-      // for eager deserialization avoid extra byte array serialization
-      if (this.deserializationPolicy ==
-          DistributedCacheOperation.DESERIALIZATION_POLICY_EAGER) {
-        setValObj(DataSerializer.readObject(in));
-      }
-      else {
-        setValBytes(DataSerializer.readByteArray(in));
-      }
+      setValBytes(DataSerializer.readByteArray(in));
       if ((extraFlags & HAS_DELTA_WITH_FULL_VALUE) != 0) {
         this.deltaBytes = DataSerializer.readByteArray(in);
       }
@@ -642,10 +625,6 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
     if ((flags & HAS_VERSION_TAG) != 0) {
       this.versionTag =  DataSerializer.readObject(in);
     }
-    if ((flags & IS_PUT_DML) != 0) {
-      this.isPutDML = true;
-    }
-    
   }
   
   @Override
@@ -752,7 +731,6 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
       }
     }
     if (this.versionTag != null) s |= HAS_VERSION_TAG;
-    if (this.event.isPutDML()) s |= IS_PUT_DML;
     return s;
   }
 
@@ -788,9 +766,6 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
     if (eventSender == null) {
        eventSender = getSender();
     }
-    if (r.keyRequiresRegionContext()) {
-      ((KeyWithRegionContext)this.key).setRegionContext(r);
-    }
     @Released final EntryEventImpl ev = EntryEventImpl.create(
         r,
         getOperation(),
@@ -814,7 +789,6 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
     ev.setCausedByMessage(this);
     ev.setInvokePRCallbacks(!notificationOnly);
     ev.setPossibleDuplicate(this.posDup);
-    ev.setPutDML(this.isPutDML);
     /*if (this.hasOldValue) {
       if (this.oldValueIsSerialized) {
         ev.setSerializedOldValue(getOldValueBytes());
@@ -839,10 +813,6 @@ public final class PutMessage extends PartitionMessageWithDirectReply implements
         case DistributedCacheOperation.DESERIALIZATION_POLICY_NONE:
           ev.setNewValue(getValBytes());
           break;
-        case DistributedCacheOperation.DESERIALIZATION_POLICY_EAGER:
-          // new value is a Delta
-          ev.setNewValue(this.valObj); // sets the delta field
-          break;
         default:
           throw new AssertionError("unknown deserialization policy: "
               + deserializationPolicy);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RegionAdvisor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RegionAdvisor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RegionAdvisor.java
index 176a41a..a5b4d71 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RegionAdvisor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RegionAdvisor.java
@@ -1157,23 +1157,6 @@ public class RegionAdvisor extends CacheDistributionAdvisor
     return result;
   }
 
-  // For SQLFabric ALTER TABLE, need to reset the parentAdvisors if colocated
-  // region changes
-  public void resetBucketAdvisorParents() {
-    if (this.buckets != null) {
-      for (ProxyBucketRegion pbr : this.buckets) {
-        if (pbr.getCreatedBucketRegion() != null) {
-          throw new InternalGemFireException(
-              LocalizedStrings.RegionAdvisor_CANNOT_RESET_EXISTING_BUCKET
-                  .toLocalizedString(new Object[] {
-                      pbr.getPartitionedRegion().getFullPath(),
-                      pbr.getBucketId() }));
-        }
-        pbr.getBucketAdvisor().resetParentAdvisor(pbr.getBucketId());
-      }
-    }
-  }
-
   /**
    * Returns the bucket identified by bucketId after waiting for initialization
    * to finish processing queued profiles. Call synchronizes and waits on 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoteFetchKeysMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoteFetchKeysMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoteFetchKeysMessage.java
index ac95e03..bea08fc 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoteFetchKeysMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoteFetchKeysMessage.java
@@ -49,7 +49,6 @@ import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.HeapDataOutputStream;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
 import com.gemstone.gemfire.internal.cache.InitialImageOperation;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.RemoteOperationException;
@@ -421,14 +420,9 @@ public class RemoteFetchKeysMessage extends RemoteOperationMessage {
         
         ByteArrayInputStream byteStream = new ByteArrayInputStream(msg.chunk);
         DataInputStream in = new DataInputStream(byteStream);
-        final boolean requiresRegionContext = this.region
-            .keyRequiresRegionContext();
         while (in.available() > 0) {
           Object key = DataSerializer.readObject(in);
           if (key != null) {
-            if (requiresRegionContext) {
-              ((KeyWithRegionContext)key).setRegionContext(this.region);
-            }
             synchronized(returnValue) {
               returnValue.add(key);
             }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoveAllPRMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoveAllPRMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoveAllPRMessage.java
index 125cdb0..5c9e799 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoveAllPRMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/RemoveAllPRMessage.java
@@ -59,7 +59,6 @@ import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EnumListenerEvent;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.ForceReattemptException;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegionDataStore;
@@ -260,10 +259,6 @@ public final class RemoveAllPRMessage extends PartitionMessageWithDirectReply
       EntryVersionsList versionTags = new EntryVersionsList(removeAllPRDataSize);
 
       boolean hasTags = false;
-      // get the "keyRequiresRegionContext" flag from first element assuming
-      // all key objects to be uniform
-      final boolean requiresRegionContext =
-        (this.removeAllPRData[0].getKey() instanceof KeyWithRegionContext);
       for (int i = 0; i < this.removeAllPRDataSize; i++) {
         // If sender's version is >= 7.0.1 then we can send versions list.
         if (!hasTags && removeAllPRData[i].versionTag != null) {
@@ -273,7 +268,7 @@ public final class RemoveAllPRMessage extends PartitionMessageWithDirectReply
         VersionTag<?> tag = removeAllPRData[i].versionTag;
         versionTags.add(tag);
         removeAllPRData[i].versionTag = null;
-        removeAllPRData[i].toData(out, requiresRegionContext);
+        removeAllPRData[i].toData(out);
         removeAllPRData[i].versionTag = tag;
         // RemoveAllEntryData's toData did not serialize eventID to save
         // performance for DR, but in PR,
@@ -423,12 +418,8 @@ public final class RemoveAllPRMessage extends PartitionMessageWithDirectReply
     // Fix the updateMsg misorder issue
     // Lock the keys when doing postRemoveAll
     Object keys[] = new Object[removeAllPRDataSize];
-    final boolean keyRequiresRegionContext = r.keyRequiresRegionContext();
     for (int i = 0; i < removeAllPRDataSize; ++i) {
       keys[i] = removeAllPRData[i].getKey();
-      if (keyRequiresRegionContext) {
-        ((KeyWithRegionContext)keys[i]).setRegionContext(r);
-      }
     }
 
     if (!notificationOnly) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/rebalance/PartitionedRegionLoadModel.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/rebalance/PartitionedRegionLoadModel.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/rebalance/PartitionedRegionLoadModel.java
index 6111561..44647b5 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/rebalance/PartitionedRegionLoadModel.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/rebalance/PartitionedRegionLoadModel.java
@@ -243,8 +243,7 @@ public class PartitionedRegionLoadModel {
         // [sumedh] remove from buckets array too to be consistent since
         // this method will be invoked repeatedly for all colocated regions,
         // and then we may miss some colocated regions for a bucket leading
-        // to all kinds of issues later (e.g. see SQLF test for #41472 that
-        //   shows some problems including NPEs, hangs etc.)
+        // to all kinds of issues later
         this.buckets[i] = null;
         continue;
       }
@@ -555,22 +554,12 @@ public class PartitionedRegionLoadModel {
   public Move findBestPrimaryMove() {
     Move bestMove= null;
     double bestImprovement = 0;
-    GemFireCacheImpl cache = null;
     for(Member source: this.members.values()) {
       for(Bucket bucket: source.getPrimaryBuckets()) {
         for(Member target: bucket.getMembersHosting()) {
           if(source.equals(target)) {
             continue;
           }
-          // If node is not fully initialized yet, then skip this node
-          // (SQLFabric DDL replay in progress).
-          if (cache == null) {
-            cache = GemFireCacheImpl.getInstance();
-          }
-          if (cache != null
-              && cache.isUnInitializedMember(target.getMemberId())) {
-            continue;
-          }
           double improvement = improvement(source.getPrimaryLoad(), source
               .getWeight(), target.getPrimaryLoad(), target.getWeight(), bucket.getPrimaryLoad(),
               getPrimaryAverage());
@@ -1211,12 +1200,6 @@ public class PartitionedRegionLoadModel {
       if(getBuckets().contains(bucket)) {
         return RefusalReason.ALREADY_HOSTING;
       }
-      // If node is not fully initialized yet, then skip this node (SQLFabric
-      // DDL replay in progress).
-      final GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
-      if (cache != null && cache.isUnInitializedMember(getMemberId())) {
-        return RefusalReason.UNITIALIZED_MEMBER;
-      }
       //Check the ip address
       if(checkZone) {
         //If the source member is equivalent to the target member, go

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java
index 448be92..e3a3d53 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GatewayReceiverCommand.java
@@ -39,7 +39,6 @@ import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.EventIDHolder;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.KeyWithRegionContext;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
@@ -320,9 +319,6 @@ public class GatewayReceiverCommand extends BaseCommand {
             try {
               byte[] value = valuePart.getSerializedForm();
               boolean isObject = valuePart.isObject();
-              if (region.keyRequiresRegionContext()) {
-                ((KeyWithRegionContext)key).setRegionContext(region);
-              }
               // [sumedh] This should be done on client while sending
               // since that is the WAN gateway
               AuthorizeRequest authzRequest = servConn.getAuthzRequest();
@@ -428,9 +424,6 @@ public class GatewayReceiverCommand extends BaseCommand {
             try {
               byte[] value = valuePart.getSerializedForm();
               boolean isObject = valuePart.isObject();
-              if (region.keyRequiresRegionContext()) {
-                ((KeyWithRegionContext)key).setRegionContext(region);
-              }
               AuthorizeRequest authzRequest = servConn.getAuthzRequest();
               if (authzRequest != null) {
                 PutOperationContext putContext = authzRequest.putAuthorize(
@@ -523,9 +516,6 @@ public class GatewayReceiverCommand extends BaseCommand {
             }
             handleMessageRetry(region, clientEvent);
             // Destroy the entry
-            if (region.keyRequiresRegionContext()) {
-              ((KeyWithRegionContext)key).setRegionContext(region);
-            }
             try {
               AuthorizeRequest authzRequest = servConn.getAuthzRequest();
               if (authzRequest != null) {
@@ -606,9 +596,6 @@ public class GatewayReceiverCommand extends BaseCommand {
               }
               
               // Update the version tag
-              if (region.keyRequiresRegionContext()) {
-                ((KeyWithRegionContext) key).setRegionContext(region);
-              }
               try {
 
                 region.basicBridgeUpdateVersionStamp(key, callbackArg, servConn.getProxyID(), false, clientEvent);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/880f8648/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tx/DistTxEntryEvent.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tx/DistTxEntryEvent.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tx/DistTxEntryEvent.java
index 6e7c21c..c84fb99 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tx/DistTxEntryEvent.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tx/DistTxEntryEvent.java
@@ -138,12 +138,7 @@ public class DistTxEntryEvent extends EntryEventImpl {
     EntryVersionsList versionTags = new EntryVersionsList(
         this.putAllOp.putAllDataSize);
     boolean hasTags = false;
-    // get the "keyRequiresRegionContext" flag from first element assuming
-    // all key objects to be uniform
     final PutAllEntryData[] putAllData = this.putAllOp.getPutAllEntryData();
-    // final boolean requiresRegionContext =
-    // (putAllData[0].key instanceof KeyWithRegionContext);
-    final boolean requiresRegionContext = false;
     for (int i = 0; i < this.putAllOp.putAllDataSize; i++) {
       if (!hasTags && putAllData[i].versionTag != null) {
         hasTags = true;
@@ -151,7 +146,7 @@ public class DistTxEntryEvent extends EntryEventImpl {
       VersionTag<?> tag = putAllData[i].versionTag;
       versionTags.add(tag);
       putAllData[i].versionTag = null;
-      putAllData[i].toData(out, requiresRegionContext);
+      putAllData[i].toData(out);
       putAllData[i].versionTag = tag;
     }
     out.writeBoolean(hasTags);
@@ -206,13 +201,8 @@ public class DistTxEntryEvent extends EntryEventImpl {
         this.removeAllOp.removeAllDataSize);
   
     boolean hasTags = false;
-    // get the "keyRequiresRegionContext" flag from first element assuming
-    // all key objects to be uniform
-    // final boolean requiresRegionContext =
-    // (this.removeAllData[0].key instanceof KeyWithRegionContext);
     final RemoveAllEntryData[] removeAllData = this.removeAllOp
         .getRemoveAllEntryData();
-    final boolean requiresRegionContext = false;
     for (int i = 0; i < this.removeAllOp.removeAllDataSize; i++) {
       if (!hasTags && removeAllData[i].versionTag != null) {
         hasTags = true;
@@ -220,7 +210,7 @@ public class DistTxEntryEvent extends EntryEventImpl {
       VersionTag<?> tag = removeAllData[i].versionTag;
       versionTags.add(tag);
       removeAllData[i].versionTag = null;
-      removeAllData[i].toData(out, requiresRegionContext);
+      removeAllData[i].toData(out);
       removeAllData[i].versionTag = tag;
     }
     out.writeBoolean(hasTags);