You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2016/05/04 22:57:02 UTC

[02/63] [abbrv] incubator-geode git commit: GEODE-1072: Removing HDFS related code

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
index e6c0b60..f344938 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
@@ -87,7 +87,6 @@ import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueueFactory;
 import com.gemstone.gemfire.cache.client.ClientCache;
 import com.gemstone.gemfire.cache.client.PoolFactory;
 import com.gemstone.gemfire.cache.execute.Function;
-import com.gemstone.gemfire.cache.hdfs.internal.HDFSStoreCreation;
 import com.gemstone.gemfire.cache.partition.PartitionListener;
 import com.gemstone.gemfire.cache.query.IndexType;
 import com.gemstone.gemfire.cache.query.internal.index.IndexCreationData;
@@ -1020,161 +1019,7 @@ public class CacheXmlParser extends CacheXml implements ContentHandler {
 
     stack.push(attrs);
   }
-  /**
-   * When a <code>hdfs-store</code> element is first encountered, we
-   * create a {@link HDFSStoreCreation}, populate it accordingly, and
-   * push it on the stack.
-   * <pre>
-   * {@code
-   * <hdfs-store name="" gemfire-home-dir="" namenode-url="" hdfs-client-config-file="">
-   * ...
-   * </hdfs-store>
-   * }
-   * 
-   */
-  private void startHDFSStore(Attributes atts) {
-    // this is the only place to create DSAC objects
-    HDFSStoreCreation attrs = new HDFSStoreCreation();
-    String name = atts.getValue(NAME);
-    if (name == null) {
-      throw new InternalGemFireException(
-          LocalizedStrings.CacheXmlParser_NULL_DiskStoreName.toLocalizedString());
-    } else {
-      attrs.setName(name);
-    }
 
-    String namenode = atts.getValue(HDFS_NAMENODE_URL);
-    if (namenode == null) {
-      throw new InternalGemFireException(
-          LocalizedStrings.CacheXmlParser_NULL_DiskStoreName.toLocalizedString());
-    } else {
-      attrs.setNameNodeURL(namenode);
-    }
-
-    String clientConfig = atts.getValue(HDFS_CLIENT_CONFIG_FILE);
-    if (clientConfig != null) {
-      attrs.setHDFSClientConfigFile(clientConfig);
-    }
-    
-    String folderPath = atts.getValue(HDFS_HOME_DIR);
-    if (folderPath != null) {
-      attrs.setHomeDir(folderPath);
-    }
-   
-    String readCacheSize = atts.getValue(HDFS_READ_CACHE_SIZE);
-    if (readCacheSize != null) {
-      try {
-        attrs.setBlockCacheSize(Float.valueOf(readCacheSize));
-      } catch (NumberFormatException e) {
-        throw new CacheXmlException(
-            LocalizedStrings.DistributedSystemConfigImpl_0_IS_NOT_A_VALID_INTEGER_1
-            .toLocalizedString(new Object[] { readCacheSize, HDFS_READ_CACHE_SIZE }),
-            e);
-      }
-    }
-    
-    Integer maxMemory = getIntValue(atts, HDFS_MAX_MEMORY);
-    if (maxMemory != null) {
-      attrs.setMaxMemory(maxMemory);
-    }
-    
-    Integer batchSize = getIntValue(atts, HDFS_BATCH_SIZE);
-    if (batchSize != null) {
-      attrs.setBatchSize(batchSize);
-    }
-    
-    Integer batchInterval = getIntValue(atts, HDFS_BATCH_INTERVAL);
-    if (batchInterval != null) {
-      attrs.setBatchInterval(batchInterval);
-    }
-    
-    Integer dispatcherThreads = getIntValue(atts, HDFS_DISPATCHER_THREADS);
-    if (dispatcherThreads != null) {
-      attrs.setDispatcherThreads(dispatcherThreads);
-    }
-    
-    Boolean bufferPersistent = getBoolean(atts, HDFS_BUFFER_PERSISTENT);
-    if (bufferPersistent != null) {
-      attrs.setBufferPersistent(bufferPersistent);
-    }
-    
-    Boolean synchronousDiskWrite = getBoolean(atts, HDFS_SYNCHRONOUS_DISK_WRITE);
-    if (synchronousDiskWrite != null) {
-      attrs.setSynchronousDiskWrite(synchronousDiskWrite);
-    }
-    
-    String diskstoreName = atts.getValue(HDFS_DISK_STORE);
-    if (diskstoreName != null) {
-      attrs.setDiskStoreName(diskstoreName);
-    }
-    
-    Integer purgeInterval = getInteger(atts, HDFS_PURGE_INTERVAL);
-    if (purgeInterval != null) {
-      attrs.setPurgeInterval(purgeInterval);
-    }
-    Boolean majorCompaction = getBoolean(atts, HDFS_MAJOR_COMPACTION);
-    if (majorCompaction != null) {
-      attrs.setMajorCompaction(Boolean.valueOf(majorCompaction));
-    }
-    
-    // configure major compaction interval
-    Integer majorCompactionInterval = getIntValue(atts, HDFS_MAJOR_COMPACTION_INTERVAL);
-    if (majorCompactionInterval != null) {
-      attrs.setMajorCompactionInterval(majorCompactionInterval);
-    }
-    
-    // configure compaction concurrency
-    Integer value = getIntValue(atts, HDFS_MAJOR_COMPACTION_THREADS);
-    if (value != null)
-      attrs.setMajorCompactionThreads(value);
-    
-    Boolean minorCompaction = getBoolean(atts, HDFS_MINOR_COMPACTION);
-    if (minorCompaction != null) {
-      attrs.setMinorCompaction(Boolean.valueOf(minorCompaction));
-    }
-    
-    // configure compaction concurrency
-    value = getIntValue(atts, HDFS_MINOR_COMPACTION_THREADS);
-    if (value != null)
-      attrs.setMinorCompactionThreads(value);
-    
-    String maxFileSize = atts.getValue(HDFS_MAX_WRITE_ONLY_FILE_SIZE);
-    if (maxFileSize != null) {
-      attrs.setWriteOnlyFileRolloverSize(parseInt(maxFileSize));
-    }
-    
-    String fileRolloverInterval = atts.getValue(HDFS_WRITE_ONLY_FILE_ROLLOVER_INTERVAL);
-    if (fileRolloverInterval != null) {
-      attrs.setWriteOnlyFileRolloverInterval(parseInt(fileRolloverInterval));
-    }
-    stack.push(name);
-    stack.push(attrs);
-  }
-  
-  /**
-   * After popping the current <code>HDFSStoreCreation</code> off the
-   * stack, we add it to the <code>HDFSStoreCreation</code> that should be on the
-   * top of the stack.
-   */
-  private void endHDFSStore() {
-    HDFSStoreCreation hsc = (HDFSStoreCreation) stack.pop();
-    String name = (String) stack.pop();
-    CacheCreation cache;
-    Object top = stack.peek();
-    if (top instanceof CacheCreation) {
-      cache = (CacheCreation) top;
-    }
-    else {
-      String s = "Did not expect a " + top.getClass().getName()
-          + " on top of the stack.";
-      Assert.assertTrue(false, s);
-      cache = null; // Dead code
-    }
-    if (name != null) {
-      cache.addHDFSStore(name, hsc);
-    }
-  }
-	
   private Integer getIntValue(Attributes atts, String param) {
     String maxInputFileSizeMB = atts.getValue(param);
     if (maxInputFileSizeMB != null) {
@@ -1389,16 +1234,7 @@ public class CacheXmlParser extends CacheXml implements ContentHandler {
     if(offHeapStr != null) {
       attrs.setOffHeap(Boolean.valueOf(offHeapStr).booleanValue());
     }
-    String hdfsStoreName = atts.getValue(HDFS_STORE_NAME);
-    if (hdfsStoreName != null) {
-      attrs.setHDFSStoreName(hdfsStoreName);
-    }
-    String hdfsWriteOnly= atts.getValue(HDFS_WRITE_ONLY);
-    if (hdfsWriteOnly != null) {
-      attrs.setHDFSWriteOnly(Boolean.valueOf(hdfsWriteOnly).booleanValue());
-    }
 
-    
     stack.push(attrs);
   }
   
@@ -3000,9 +2836,6 @@ public class CacheXmlParser extends CacheXml implements ContentHandler {
     } else if(qName.equals(PDX_SERIALIZER)) {
       //do nothing
     }
-	else if (qName.equals(HDFS_STORE)) {
-        startHDFSStore(atts);
-    }
     else if (qName.equals(COMPRESSOR)) {
     }
     else {
@@ -3411,9 +3244,6 @@ public class CacheXmlParser extends CacheXml implements ContentHandler {
       else if (qName.equals(PDX_SERIALIZER)) {
         endPdxSerializer();
       }
-      else if (qName.equals(HDFS_STORE)) {
-          endHDFSStore();
-      }
       else if (qName.equals(COMPRESSOR)) {
         endCompressor();
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/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 d0f5676..4dfe6ae 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
@@ -28,7 +28,6 @@ import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheListener;
 import com.gemstone.gemfire.cache.CacheLoader;
 import com.gemstone.gemfire.cache.CacheWriter;
-import com.gemstone.gemfire.cache.CustomEvictionAttributes;
 import com.gemstone.gemfire.cache.CustomExpiry;
 import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.DiskStoreFactory;
@@ -123,8 +122,6 @@ public class RegionAttributesCreation extends UserSpecifiedRegionAttributes impl
   * @since prPersistPrint2 
   * */
   private String diskStoreName;
-  private String hdfsStoreName;
-  private boolean hdfsWriteOnly = false;
   private boolean isDiskSynchronous = AttributesFactory.DEFAULT_DISK_SYNCHRONOUS;
   
   private boolean cloningEnabled = false;
@@ -271,8 +268,7 @@ public class RegionAttributesCreation extends UserSpecifiedRegionAttributes impl
     this.poolName = attrs.getPoolName();
     this.multicastEnabled = attrs.getMulticastEnabled();
     this.cloningEnabled = attrs.getCloningEnabled();
-	this.hdfsStoreName = attrs.getHDFSStoreName();
-    
+
     this.compressor = attrs.getCompressor();
     this.offHeap = attrs.getOffHeap();
     if (attrs instanceof UserSpecifiedRegionAttributes) {
@@ -500,10 +496,6 @@ public class RegionAttributesCreation extends UserSpecifiedRegionAttributes impl
     if(this.cloningEnabled != other.getCloningEnabled()){
       throw new RuntimeException(LocalizedStrings.RegionAttributesCreation__CLONING_ENABLE_IS_NOT_THE_SAME_THIS_0_OTHER_1.toLocalizedString(new Object[] {Boolean.valueOf(this.cloningEnabled), Boolean.valueOf(other.getCloningEnabled())}));
     }
- 	if (! equal(this.hdfsStoreName, other.getHDFSStoreName())) {
-      //TODO:HDFS write a new exception string
-      throw new RuntimeException(" HDFS Store name does not match");
-    }
     if(! equal(this.compressor, other.getCompressor())) {
       throw new RuntimeException("Compressors are not the same.");
     }
@@ -1448,25 +1440,7 @@ public class RegionAttributesCreation extends UserSpecifiedRegionAttributes impl
         setDiskSynchronous(parent.isDiskSynchronous());
       }
     }
-    if (!hasHDFSStoreName()) {
-      if (parentIsUserSpecified) {
-        if (parentWithHas.hasHDFSStoreName()) {
-          setHDFSStoreName(parent.getHDFSStoreName());
-        }
-      } else {
-        setHDFSStoreName(parent.getHDFSStoreName());
-      }
-    }
-    if (!hasHDFSWriteOnly()) {
-      if (parentIsUserSpecified) {
-        if (parentWithHas.hasHDFSWriteOnly()) {
-          setHDFSWriteOnly(parent.getHDFSWriteOnly());
-        }
-      } else {
-        setHDFSWriteOnly(parent.getHDFSWriteOnly());
-      }
-    }
-    
+
     if(!hasCompressor()) {
       if (parentIsUserSpecified) {
         if (parentWithHas.hasCompressor()) {
@@ -1554,15 +1528,6 @@ public class RegionAttributesCreation extends UserSpecifiedRegionAttributes impl
     return this.evictionAttributes;
   }
 
-  /**
-   * {@inheritDoc}
-   */
-  @Override
-  public CustomEvictionAttributes getCustomEvictionAttributes() {
-    // TODO: HDFS: no support for configuring this from XML yet
-    return null;
-  }
-
   public void setPoolName(String poolName) {
     if ("".equals(poolName)) {
       poolName = null;
@@ -1655,20 +1620,4 @@ public class RegionAttributesCreation extends UserSpecifiedRegionAttributes impl
   public Set<String> getGatewaySenderIds() {
     return this.gatewaySenderIds;
   }
-  public String getHDFSStoreName() {
-    return this.hdfsStoreName;
-  }
-  public void setHDFSStoreName(String hdfsStoreName) {
-    //TODO:HDFS : throw an exception if a disk store is already configured
-    // and vice versa
-    this.hdfsStoreName = hdfsStoreName;
-    setHasHDFSStoreName(true);
-  }
-  public void setHDFSWriteOnly(boolean writeOnly) {
-    this.hdfsWriteOnly= writeOnly;
-    setHasHDFSWriteOnly(true);
-  }
-  public boolean getHDFSWriteOnly() {
-    return hdfsWriteOnly;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/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 2a939b4..ff960ca 100644
--- 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
@@ -1997,32 +1997,6 @@ public class LocalizedStrings extends ParentLocalizedStrings {
   public static final StringId SnappyCompressor_UNABLE_TO_LOAD_NATIVE_SNAPPY_LIBRARY = new StringId(5502, "Unable to load native Snappy library.");
   public static final StringId SnappyCompressor_UNABLE_TO_LOAD_NATIVE_SNAPPY_LIBRARY_MISSING_LIBRARY = new StringId(5503, "Unable to load native Snappy library from: {0}");
   
-  /** HOPLOG STRINGS, 5505 - 5600 **/
-  public static final StringId HOPLOG_REGION_CLOSE_FAILED = new StringId(5505, "IO error while trying to close region and release hdfs connection: {0}");
-  public static final StringId HOPLOG_HDFS_CLIENT_CONFIG_FILE_ABSENT = new StringId(5506, "HDFS client config file does not exist: {0}");
-  public static final StringId HOPLOG_IO_ERROR = new StringId(5507, "IO Exception while executing HDFS operation: {0}");
-  public static final StringId HOPLOG_UNABLE_TO_DELETE_FILE = new StringId(5508, "Unable to delete file: {0}");
-  public static final StringId HOPLOG_UNABLE_TO_DELETE_HDFS_DATA = new StringId(5509, "Unable to delete HDFS data while destroying region");
-  public static final StringId HOPLOG_CLOSE_FAILED = new StringId(5510, "IO error while trying to close hoplog.");
-  public static final StringId HOPLOG_FLUSH_FOR_BATCH_FAILED = new StringId(5511, "A batch of data could not be persisted on HDFS. It will be retried.");
-  public static final StringId HOPLOG_HDFS_STORE_NOT_FOUND = new StringId(5512, "HDFS store ''{0}'' does not exist.");
-  public static final StringId HOPLOG_TRYING_TO_CREATE_STANDALONE_SYSTEM = new StringId(5513, "The namenode url {0} is not valid. Please use the format hdfs://HOST:PORT");
-  public static final StringId HOPLOG_DOES_NOT_USE_HDFSSTORE = new StringId(5514, "{0} does not use HDFSSTORE");
-  public static final StringId HOPLOG_CONFIGURED_AS_WRITEONLY = new StringId(5515, "{0} is defined as WRITEONLY");
-  public static final StringId HOPLOG_MISSING_IN_BUCKET_FORCED_CLOSED = new StringId(5516, "A hoplog file, {0}, was not found in bucket lists. Closing it now, it may impact active reads.");
-  public static final StringId HOPLOG_MIN_IS_MORE_THAN_MAX = new StringId(5517, "Value of {0} is {1}. It should not be more than {2} value {3}");
-  public static final StringId HOPLOG_NOT_STARTED_YET = new StringId(5518, "HDFS store is not started yet. Gemfire is running without HDFS.");
-  public static final StringId HOPLOG_0_COLOCATE_WITH_REGION_1_NOT_INITIALIZED_YET = new StringId(5519, "Current region: {0} colocated with region {1} is yet initialized.");
-  public static final StringId HOPLOG_SUSPEND_OF_0_FAILED_IN_1 = new StringId(5520, "Failed to suspend active {0} in {1}");
-  public static final StringId HOPLOG_CLEANED_UP_BY_JANITOR = new StringId(5521, "Hoplog is cleaned up by janitor task.");
-  public static final StringId HOPLOG_HDFS_UNREACHABLE = new StringId(5522, "HDFS at {0} is unreachable.");
-  public static final StringId HOPLOG_MAJOR_COMPACTION_SCHEDULED_FOR_BETTER_ESTIMATE = new StringId(5523, "A major compaction has been automatically scheduled for better accuracy of count_estimate() function");
-  public static final StringId HOPLOG_FAILED_TO_READ_HDFS_FILE = new StringId(5524, "Exception while reading file on HDFS: {0}");
-  public static final StringId HOPLOG_HDFS_COMPACTION_ERROR = new StringId(5525, "Error while compacting files of bucket {0}");
-  public static final StringId HOPLOG_HDFS_COMPACTION_OVERLOADED = new StringId(5526, "Too many pending tasks for {0}. Skipping compaction request for {1}");
-  public static final StringId HOPLOG_FLUSH_OPERATION_FAILED = new StringId(5527, "IO error while trying to flush buffer and create hoplog.");
-  public static final StringId HOPLOG_HOPLOG_REMOVE_FAILED = new StringId(5528, "IO error while trying to remove hoplog.");
-  /** HOPLOG STRINGS, 5505 - 5600 **/
 
   public static final StringId PartitionAttributesImpl_CANNOT_DETERMINE_LOCAL_MAX_MEMORY_FOR_PARTITION_ATTRIBUTE_SINCE_NO_CACHE_IS_AVAILABLE_FROM_WHICH_TO_FETCH_THE_OFF_HEAP_MEMORY_ALLOCATOR = new StringId(5600, "Cannot determine local max memory for partition attribute since no cache is available from which to fetch the off-heap memory allocator");
 
@@ -2096,10 +2070,6 @@ public class LocalizedStrings extends ParentLocalizedStrings {
   public static final StringId ParallelAsyncEventQueue_0_CAN_NOT_BE_USED_WITH_REPLICATED_REGION_1 = new StringId(5716,"Parallel Async Event Queue {0} can not be used with replicated region {1}");
   public static final StringId ParallelGatewaySender_0_CAN_NOT_BE_USED_WITH_REPLICATED_REGION_1 = new StringId(5717,"Parallel gateway sender {0} can not be used with replicated region {1}");
 
-  public static final StringId HDFSSTORE_IS_USED_IN_NONHDFS_REGION = new StringId(5808, "Only regions with HDFS_PARTITION or HDFS_PERSISTENT_PARTITION data policies can specify a HDFS Store");
-  public static final StringId EVICTORSERVICE_CAUGHT_EXCEPTION_0 = new StringId(5809, "Evictor Service caught following exception : {0}");
-  public static final StringId HDFSSTORE_IS_USED_IN_REPLICATED_TABLE = new StringId(5810, "HDFS Store cannot be used for REPLICATED TABLE");
-  public static final StringId HDFS_USER_IS_SAME_AS_GF_USER = new StringId(5811, "Gemfire user is the same as HDFS user, may cause security risks: {0}");
   public static final StringId GF_KERBEROS_KEYTAB_FILE_ABSENT = new StringId(5812, "Gemfire kerberos keytab file is missing: {0}");
   public static final StringId GF_KERBEROS_NAMENODE_PRINCIPAL_UNDEF = new StringId(5813, "Namenode principal must be configured when using kerberos authentication");
   public static final StringId GF_KERBEROS_KEYTAB_UNDEF = new StringId(5814, "Gemfire kerberos keytab file is not configured");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedRegionMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedRegionMXBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedRegionMXBean.java
index 3003827..f087c89 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedRegionMXBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedRegionMXBean.java
@@ -130,7 +130,6 @@ public interface DistributedRegionMXBean {
   /**
    * Returns the number of entries in the Region.
    * 
-   * For HDFS regions it will be count of only in memory data.
    */
   public long getSystemRegionEntryCount();
 
@@ -305,14 +304,4 @@ public interface DistributedRegionMXBean {
    * Returns the number of members whose entry count is 0.
    */
   public int getEmptyNodes();
-  
-  
-  /**
-   * An estimated entry count for HDFS Read-Write region.This may not be accurate but acts
-   * as an indicative value.
-   * 
-   * For other regions it will be -1 ( Not Available)
-   */
-  public long getEstimatedSizeForHDFSRegion();
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedSystemMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedSystemMXBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedSystemMXBean.java
index a6f65d4..88c4058 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedSystemMXBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/DistributedSystemMXBean.java
@@ -130,14 +130,6 @@ public interface DistributedSystemMXBean {
    */
   public Map<String, String[]> listMemberDiskstore();
 
-  
-  /**
-   *  @return A map of all {@link DistributedMember}s and their HDFSStore's.
-   */
-  
-  public Map<String, String[]> listMemberHDFSStore();
-  
-  
   /**
    * Returns a list of IDs for all gateway senders.
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/MemberMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/MemberMXBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/MemberMXBean.java
index ed27569..4b849e0 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/MemberMXBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/MemberMXBean.java
@@ -227,13 +227,6 @@ public interface MemberMXBean {
   public String[] listDiskStores(boolean includeRegionOwned);
 
   /**
-   * 
-   * @return  list of HDFSStore's present in the Cache
-   */
-  
-  public String[] getHDFSStores();
-
-  /**
    * Returns the GemFire specific properties for this member.
    */
   public GemFireProperties listGemFireProperties();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/RegionMXBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/RegionMXBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/RegionMXBean.java
index 8c11d00..a913105 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/RegionMXBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/RegionMXBean.java
@@ -127,8 +127,6 @@ public interface RegionMXBean {
    * partitioned regions it will be the entry count for the primary buckets
    * hosted within this member.
    *
-   * For HDFS regions it will be count of only in memory data.
-   * 
    */
   public long getEntryCount();
 
@@ -350,12 +348,4 @@ public interface RegionMXBean {
    */
   public int getLocalMaxMemory();
   
-  /**
-   * Estimated entry count for HDFS Read-Write regions.This may not be accurate but
-   * acts as an indicative value. All HDFS Read-Write regions regions are PartitionedRegions. Hence
-   * the estimated value will be for primary buckets hosted within the member.
-   * 
-   * For other regions it will be -1 ( Not Available)
-   */
-  public long getEstimatedSizeForHDFSRegion();
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedRegionBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedRegionBridge.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedRegionBridge.java
index 5fbbc61..48b899b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedRegionBridge.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedRegionBridge.java
@@ -674,9 +674,4 @@ public class DistributedRegionBridge {
       return false;
     }
   }
-  
-  public long getEstimatedSizeForHDFSRegion() {
-    return monitor.getEstimatedSizeForHDFSRegion();
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedRegionMBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedRegionMBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedRegionMBean.java
index 549acc7..4580e7f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedRegionMBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedRegionMBean.java
@@ -321,9 +321,4 @@ public class DistributedRegionMBean implements DistributedRegionMXBean {
     return bridge.getEntrySize();
   }
 
-  @Override
-  public long getEstimatedSizeForHDFSRegion() {
-    return bridge.getEstimatedSizeForHDFSRegion();
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemBridge.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemBridge.java
index bcacc41..632415a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemBridge.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemBridge.java
@@ -821,25 +821,6 @@ public class DistributedSystemBridge {
     return Collections.emptyMap();
   }
   
-  
-  /**
-   *  @return A map of all {@link DistributedMember}s and their HDFSStore's.
-   */  
-  
-  public Map<String, String[]> getMemberHDFSStoreMap() {
-    Iterator<MemberMXBean> memberIterator = mapOfMembers.values().iterator();    
-    if (memberIterator != null) {
-      Map<String, String[]> mapOfHdfs = new HashMap<String, String[]>();
-      while (memberIterator.hasNext()) {
-        MemberMXBean bean = memberIterator.next();
-        mapOfHdfs.put(bean.getMember(), bean.getHDFSStores());
-      }
-
-      return mapOfHdfs;
-    }
-    return Collections.emptyMap();
-  }
-
   /**
    *
    * @param member

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemMBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemMBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemMBean.java
index bd92f9f..3458bf5 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemMBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/DistributedSystemMBean.java
@@ -450,11 +450,4 @@ public class DistributedSystemMBean extends NotificationBroadcasterSupport
   public void setQueryCollectionsDepth(int queryCollectionsDepth) {
     bridge.setQueryCollectionsDepth(queryCollectionsDepth);;
   }
-
-  @Override
-  public Map<String, String[]> listMemberHDFSStore() {
-    return bridge.getMemberHDFSStoreMap();
-  }
-
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/HDFSRegionBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/HDFSRegionBridge.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/HDFSRegionBridge.java
deleted file mode 100644
index 29bc246..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/HDFSRegionBridge.java
+++ /dev/null
@@ -1,173 +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.management.internal.beans;
-
-import java.util.Map;
-import java.util.Set;
-
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.hdfs.internal.hoplog.HDFSRegionDirector;
-import com.gemstone.gemfire.internal.cache.BucketRegion;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion.SizeEntry;
-import com.gemstone.gemfire.internal.cache.persistence.soplog.SortedOplogStatistics;
-import com.gemstone.gemfire.management.internal.ManagementConstants;
-import com.gemstone.gemfire.management.internal.beans.stats.MBeanStatsMonitor;
-import com.gemstone.gemfire.management.internal.beans.stats.StatType;
-import com.gemstone.gemfire.management.internal.beans.stats.StatsRate;
-
-/**
- * 
- * 
- * MBean Bridge for HDFS region which is a type of Partitioned Region
- */
-public class HDFSRegionBridge<K, V> extends PartitionedRegionBridge<K, V> {
-
-  private SortedOplogStatistics soplogStats;
-
-  private MBeanStatsMonitor hdfsRegionMonitor;
-
-  private static final String WRITTEN_BYTES = "writeBytes";
-
-  private static final String READ_BYTES = "readBytes";
-
-  private static final String SCANNED_BYTES = "scanBytes";
-
-  public static final String HDFS_REGION_MONITOR = "HDFSRegionMonitor";
-
-  private StatsRate diskWritesRate;
-
-  private StatsRate diskReadsRate;
-  
-  private PartitionedRegion parRegion;
-
-  public HDFSRegionBridge(Region<K, V> region) {
-    super(region);
-
-    HDFSRegionDirector director = HDFSRegionDirector.getInstance();
-
-    String regionFullPath = region.getFullPath();
-    this.soplogStats = director.getHdfsRegionStats(regionFullPath);
-    this.hdfsRegionMonitor = new MBeanStatsMonitor(HDFS_REGION_MONITOR + "_" + regionFullPath);
-    hdfsRegionMonitor.addStatisticsToMonitor(soplogStats.getStats());
-    this.parRegion = (PartitionedRegion)region;
-    configureHDFSRegionMetrics();
-  }
-
-  private void configureHDFSRegionMetrics() {
-
-    diskWritesRate = new StatsRate(WRITTEN_BYTES, StatType.INT_TYPE, hdfsRegionMonitor);
-
-    String[] readsRates = new String[] { READ_BYTES, SCANNED_BYTES };
-
-    diskReadsRate = new StatsRate(readsRates, StatType.INT_TYPE, hdfsRegionMonitor);
-  }
-
-  
-  private long estimatedEntryCount = 0;
-  
-
-  /**
-   * Initialized skipCount to 10 as for the first time we want to compute size
-   * of HDFS region.
-   */
-  private int skipCount = 10;
-
-  /**
-   * 
-   * An estimated entry count for HDFS region.This may not be accurate but acts
-   * as an indicative value.
-   * 
-   * 
-   * Even for estimating size we need to iterate over all BucketRegions and call
-   * BucketRegion.size(). This is expensive as compared to reading directly from
-   * a statistics value. Hence we are skipping 10 samples.
-   * 
-   */
-  public long getEstimatedSizeForHDFSRegion() {
-    if(parRegion.isHDFSReadWriteRegion()){
-      if(skipCount % 10 == 0) {
-        computeEntryCount();
-        skipCount = 1;
-      } else {
-        skipCount++;
-      }
-      return estimatedEntryCount;
-    }else{
-      return ManagementConstants.NOT_AVAILABLE_LONG;
-    }
-    
-  }
-  
-  private void computeEntryCount() {
-
-    if (parRegion.isDataStore()) { //if not a DataStore do nothing and keep the entryCount as 0;
-      int numLocalEntries = 0;
-      Map<Integer, SizeEntry> localPrimaryBucketRegions = parRegion.getDataStore()
-          .getSizeEstimateForLocalPrimaryBuckets();
-      if (localPrimaryBucketRegions != null && localPrimaryBucketRegions.size() > 0) {
-        for (Map.Entry<Integer, SizeEntry> me : localPrimaryBucketRegions.entrySet()) {
-          numLocalEntries += me.getValue().getSize();
-
-        }
-      }
-      this.estimatedEntryCount = numLocalEntries;
-    }
-  }
-  
-  @Override
-  public long getEntryCount() {
-    if (parRegion.isDataStore()) {
-      int numLocalEntries = 0;
-      Set<BucketRegion> localPrimaryBucketRegions = parRegion.getDataStore().getAllLocalPrimaryBucketRegions();
-      if (localPrimaryBucketRegions != null && localPrimaryBucketRegions.size() > 0) {
-        for (BucketRegion br : localPrimaryBucketRegions) {
-          // TODO soplog, fix this for griddb regions
-          numLocalEntries += br.getRegionMap().sizeInVM() - br.getTombstoneCount();
-
-        }
-      }
-      return numLocalEntries;
-    } else {
-      return  ManagementConstants.ZERO;
-    }
-  }
-
-
-  @Override
-  public long getEntrySize() {
-    return ManagementConstants.NOT_AVAILABLE_LONG;
-  }
-
-  @Override
-  public long getDiskUsage() {
-    if (soplogStats != null) {
-      return soplogStats.getStoreUsageBytes();
-    }
-    return ManagementConstants.NOT_AVAILABLE_LONG;
-  }
-
-  @Override
-  public float getDiskReadsRate() {
-    return diskReadsRate.getRate();
-  }
-
-  @Override
-  public float getDiskWritesRate() {
-    return diskWritesRate.getRate();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBean.java
index 21d7140..b82b94d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBean.java
@@ -455,11 +455,6 @@ public class MemberMBean extends NotificationBroadcasterSupport implements
   }
 
   @Override
-  public String[] getHDFSStores() {
-    return bridge.getHDFSStores();
-  }
-  
-  @Override
   public long getGetsAvgLatency() {
     return bridge.getGetsAvgLatency();
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
index 1425572..638ba06 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/MemberMBeanBridge.java
@@ -49,7 +49,6 @@ import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.cache.DiskStore;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.execute.FunctionService;
-import com.gemstone.gemfire.cache.hdfs.internal.HDFSStoreImpl;
 import com.gemstone.gemfire.cache.persistence.PersistentID;
 import com.gemstone.gemfire.cache.wan.GatewayReceiver;
 import com.gemstone.gemfire.cache.wan.GatewaySender;
@@ -1010,32 +1009,6 @@ public class MemberMBeanBridge {
     return listDiskStores(true);
   }
 
-  
-
-  
-  /**
-   * @return list all the HDFSStore's name at cache level
-   */
-  
-  public String[] getHDFSStores() {
-    GemFireCacheImpl cacheImpl = (GemFireCacheImpl) cache;
-    String[] retStr = null;
-    Collection<HDFSStoreImpl> hdfsStoreCollection = null;
-    hdfsStoreCollection = cacheImpl.getHDFSStores();
-      
-    if (hdfsStoreCollection != null && hdfsStoreCollection.size() > 0) {
-      retStr = new String[hdfsStoreCollection.size()];
-      Iterator<HDFSStoreImpl> it = hdfsStoreCollection.iterator();
-      int i = 0;
-      while (it.hasNext()) {
-        retStr[i] = it.next().getName();
-        i++;
-
-      }
-    }
-    return retStr;
-  }
-      
   /**
    * 
    * @return log of the member.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/PartitionedRegionBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/PartitionedRegionBridge.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/PartitionedRegionBridge.java
index 3a8440a..7450746 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/PartitionedRegionBridge.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/PartitionedRegionBridge.java
@@ -76,14 +76,7 @@ public class PartitionedRegionBridge<K, V>  extends RegionMBeanBridge<K, V> {
   
   
   public static <K, V> PartitionedRegionBridge<K, V> getInstance(Region<K, V> region) {
-
-    if (region.getAttributes().getDataPolicy().withHDFS()) {
-      PartitionedRegionBridge<K, V> bridge = new HDFSRegionBridge<K, V>(region);
-      return bridge;
-    } else {
-      return new PartitionedRegionBridge<K, V> (region);
-    }
-
+    return new PartitionedRegionBridge<K, V> (region);
   }
   
   
@@ -309,8 +302,4 @@ public class PartitionedRegionBridge<K, V>  extends RegionMBeanBridge<K, V> {
   public int getLocalMaxMemory() {
     return partitionAttributesData.getLocalMaxMemory();
   }
-
-  public long getEstimatedSizeForHDFSRegion() {
-    return -1;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/RegionMBean.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/RegionMBean.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/RegionMBean.java
index 1c7dcf7..86fe73e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/RegionMBean.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/RegionMBean.java
@@ -314,9 +314,4 @@ public class RegionMBean<K, V> extends NotificationBroadcasterSupport implements
     return bridge.getLocalMaxMemory(); 
   }
 
-  @Override
-  public long getEstimatedSizeForHDFSRegion() {
-    return bridge.getEstimatedSizeForHDFSRegion();
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/RegionMBeanBridge.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/RegionMBeanBridge.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/RegionMBeanBridge.java
index cd3cb90..66f61e2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/RegionMBeanBridge.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/RegionMBeanBridge.java
@@ -590,9 +590,4 @@ public class RegionMBeanBridge<K, V> {
   public int getLocalMaxMemory() {
     return -1;
   }
-
-  
-  public long getEstimatedSizeForHDFSRegion() {
-    return -1;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/stats/RegionClusterStatsMonitor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/stats/RegionClusterStatsMonitor.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/stats/RegionClusterStatsMonitor.java
index c855171..7a4d9b4 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/stats/RegionClusterStatsMonitor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/beans/stats/RegionClusterStatsMonitor.java
@@ -111,8 +111,6 @@ public class RegionClusterStatsMonitor {
 
   private static final String PERSISTENT_ENABLED = "PersistentEnabled";
   
-  private static final String ESTIMATED_SIZE_FOR_HDFS_REGION = "EstimatedSizeForHDFSRegion";
-
   private volatile long lastAccessedTime = 0;
 
   private volatile long lastModifiedTime = 0;
@@ -192,7 +190,6 @@ public class RegionClusterStatsMonitor {
     typeMap.put(AVERAGE_READS, Float.TYPE);
     typeMap.put(AVERAGE_WRITES, Float.TYPE);
     typeMap.put(ENTRY_SIZE, Long.TYPE);
-    typeMap.put(ESTIMATED_SIZE_FOR_HDFS_REGION, Long.TYPE);
 
   }
 
@@ -336,10 +333,6 @@ public class RegionClusterStatsMonitor {
   public long getTotalEntriesOnlyOnDisk() {
     return aggregator.getLongValue(TOTAL_ENTRIES_ONLY_ON_DISK);
   }
-  
-  public long getEstimatedSizeForHDFSRegion() {
-    return aggregator.getLongValue(ESTIMATED_SIZE_FOR_HDFS_REGION);
-  }
 
   public int getAvgBucketSize() {
     int bucketNum = getBucketCount();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/domain/RegionAttributesInfo.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/domain/RegionAttributesInfo.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/domain/RegionAttributesInfo.java
index c4588f6..5a51b62 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/domain/RegionAttributesInfo.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/domain/RegionAttributesInfo.java
@@ -83,11 +83,7 @@ public class RegionAttributesInfo implements Serializable{
 	private String regionIdleTimeoutAction = ExpirationAction.INVALIDATE.toString();
 	
 	private boolean offHeap;
-	private String hdfsStoreName;
-	private Boolean hdfsWriteOnly;
-	
-	
-	
+
 	/***
 	 * Non-default-attribute map in the constructor
 	 */
@@ -183,8 +179,6 @@ public class RegionAttributesInfo implements Serializable{
 		
 		}
 		this.offHeap = ra.getOffHeap();
-		this.hdfsStoreName = ra.getHDFSStoreName();
-		this.hdfsWriteOnly = ra.getHDFSWriteOnly();
 	}
 	
 	
@@ -314,15 +308,6 @@ public class RegionAttributesInfo implements Serializable{
 	  return this.offHeap;
 	}
 	
-	public String getHdfsStoreName() {
-		return hdfsStoreName;
-	}
-
-
-	public Boolean getHdfsWriteOnly() {
-		return hdfsWriteOnly;
-	}
-	
 	@Override
 	public boolean equals(Object arg0) {
 		return super.equals(arg0);
@@ -482,10 +467,6 @@ public class RegionAttributesInfo implements Serializable{
             if (this.offHeap != RegionAttributesDefault.OFF_HEAP) {
                 nonDefaultAttributes.put(RegionAttributesNames.OFF_HEAP, Boolean.toString(this.offHeap));
              }            
-            if (this.hdfsStoreName != null ) {
-                nonDefaultAttributes.put(RegionAttributesNames.HDFSSTORE, this.hdfsStoreName);
-                nonDefaultAttributes.put(RegionAttributesNames.HDFS_WRITEONLY, Boolean.toString(this.hdfsWriteOnly));
-             }
 		}
 		return this.nonDefaultAttributes;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DescribeHDFSStoreFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DescribeHDFSStoreFunction.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DescribeHDFSStoreFunction.java
deleted file mode 100644
index e6828bc..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DescribeHDFSStoreFunction.java
+++ /dev/null
@@ -1,86 +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.management.internal.cli.functions;
-
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.execute.FunctionAdapter;
-import com.gemstone.gemfire.cache.execute.FunctionContext;
-import com.gemstone.gemfire.cache.hdfs.internal.HDFSStoreConfigHolder;
-import com.gemstone.gemfire.cache.hdfs.internal.HDFSStoreImpl;
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.internal.InternalEntity;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.InternalCache;
-import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.management.internal.cli.util.HDFSStoreNotFoundException;
-
-/**
- *  Function used by the 'describe hdfs-store' gfsh command to collect information
- * and details about a particular hdfs store for a particular GemFire distributed system member.
- * 
- */
-public class DescribeHDFSStoreFunction extends FunctionAdapter implements InternalEntity {
-  private static final long serialVersionUID = 1L;
-
-  private static final Logger logger = LogService.getLogger();
-
-  public static DescribeHDFSStoreFunction INSTANCE = new DescribeHDFSStoreFunction();
-
-  private static final String ID = DescribeHDFSStoreFunction.class.getName();
-  
-  protected Cache getCache() {
-    return CacheFactory.getAnyInstance();
-  }
-  
-  protected DistributedMember getDistributedMemberId(Cache cache){
-    return ((InternalCache)cache).getMyId();
-  }
-  
-  public void execute(final FunctionContext context) {
-    try {
-      Cache cache = getCache();
-      final DistributedMember member = getDistributedMemberId(cache);      
-      if (cache instanceof GemFireCacheImpl) {
-        GemFireCacheImpl cacheImpl = (GemFireCacheImpl)cache;
-        final String hdfsStoreName = (String)context.getArguments();
-        final String memberName = member.getName();
-        HDFSStoreImpl hdfsStore = cacheImpl.findHDFSStore(hdfsStoreName);        
-        if (hdfsStore != null) {
-          HDFSStoreConfigHolder configHolder = new HDFSStoreConfigHolder (hdfsStore);
-          context.getResultSender().lastResult(configHolder);
-        }
-        else {
-          context.getResultSender().sendException(
-              new HDFSStoreNotFoundException(
-                  String.format("A hdfs store with name (%1$s) was not found on member (%2$s).",
-                  hdfsStoreName, memberName)));
-        }
-      }  
-    } catch (Exception e) {
-      logger.error("Error occurred while executing 'describe hdfs-store': {}!", e.getMessage(), e);
-      context.getResultSender().sendException(e);
-    }
-  }
-
-  @Override
-  public String getId() {
-    return ID;
-  }	
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/HDFSStoreNotFoundException.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/HDFSStoreNotFoundException.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/HDFSStoreNotFoundException.java
deleted file mode 100644
index ad569f0..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/HDFSStoreNotFoundException.java
+++ /dev/null
@@ -1,47 +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.management.internal.cli.util;
-
-import com.gemstone.gemfire.GemFireException;
-
-/**
- * The HDFSStoreNotFoundException is a GemFireException class indicating that a hdfs store by name could not be found
- * on a member specified by name!
- * </p>
- * @see com.gemstone.gemfire.GemFireException
- */
-// TODO this GemFireException should be moved to a more appropriate package!
-  @SuppressWarnings("unused")
-public class HDFSStoreNotFoundException extends GemFireException {
-
-  public HDFSStoreNotFoundException() {
-  }
-
-  public HDFSStoreNotFoundException(final String message) {
-    super(message);
-  }
-
-  public HDFSStoreNotFoundException(final Throwable cause) {
-    super(cause);
-  }
-
-  public HDFSStoreNotFoundException(final String message, final Throwable cause) {
-    super(message, cause);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/RegionAttributesNames.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/RegionAttributesNames.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/RegionAttributesNames.java
index e842bee..e0db821 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/RegionAttributesNames.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/RegionAttributesNames.java
@@ -46,9 +46,7 @@ public class RegionAttributesNames {
 	public static final String POOL_NAME =  "pool-name";
 	public static final String COMPRESSOR = "compressor";
     public static final String OFF_HEAP = "off-heap";
-    public static final String HDFSSTORE = "hdfs-store";
-    public static final String HDFS_WRITEONLY = "hdfs-write-only";
-	
+
 	//Partition attributes
 	public static final String LOCAL_MAX_MEMORY =  "local-max-memory";
 	public static final String REDUNDANT_COPIES =  "redundant-copies";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/support/MemberMXBeanAdapter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/support/MemberMXBeanAdapter.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/support/MemberMXBeanAdapter.java
index f1e0d7a..a3d4cd0 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/support/MemberMXBeanAdapter.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/support/MemberMXBeanAdapter.java
@@ -243,11 +243,6 @@ public class MemberMXBeanAdapter implements MemberMXBean {
   }
 
   @Override
-  public String[] getHDFSStores() {
-    throw new UnsupportedOperationException("Not Implemented!");
-  }  
-  
-  @Override
   public String[] getRootRegionNames() {
     throw new UnsupportedOperationException("Not Implemented!");
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/main/resources/META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd
----------------------------------------------------------------------
diff --git a/geode-core/src/main/resources/META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd b/geode-core/src/main/resources/META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd
index 5ecd67d..cc6d189 100755
--- a/geode-core/src/main/resources/META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd
+++ b/geode-core/src/main/resources/META-INF/schemas/geode.apache.org/schema/cache/cache-1.0.xsd
@@ -274,7 +274,6 @@ declarative caching XML file elements unless indicated otherwise.
         </xsd:element>
         <xsd:element maxOccurs="unbounded" minOccurs="0" name="pool" type="gf:pool-type" />
         <xsd:element maxOccurs="unbounded" minOccurs="0" name="disk-store" type="gf:disk-store-type" />
-        <xsd:element maxOccurs="unbounded" minOccurs="0" name="hdfs-store" type="gf:hdfs-store-type" />
         <xsd:element maxOccurs="1" minOccurs="0" name="pdx" type="gf:pdx-type" />
         <xsd:element maxOccurs="unbounded" minOccurs="0" name="region-attributes" type="gf:region-attributes-type" />
         <xsd:choice maxOccurs="unbounded" minOccurs="0">
@@ -826,8 +825,6 @@ As of 6.5 disk-dirs is deprecated on region-attributes. Use disk-store-name inst
     <xsd:attribute name="pool-name" type="xsd:string" use="optional" />
     <xsd:attribute name="disk-store-name" type="xsd:string" use="optional" />
     <xsd:attribute name="disk-synchronous" type="xsd:boolean" use="optional" />
-    <xsd:attribute name="hdfs-store-name" type="xsd:string" use="optional" />
-    <xsd:attribute name="hdfs-write-only" type="xsd:boolean" use="optional" />
     <xsd:attribute name="publisher" type="xsd:boolean" use="optional" />
     <xsd:attribute name="refid" type="xsd:string" use="optional" />
     <xsd:attribute name="scope" type="gf:region-attributesScope" use="optional" />
@@ -1133,34 +1130,6 @@ As of 6.5 disk-dirs is deprecated on region-attributes. Use disk-store-name inst
     <xsd:attribute name="disk-usage-critical-percentage" type="xsd:string" use="optional" />
   </xsd:complexType>
 
-  <xsd:complexType mixed="true" name="hdfs-store-type">
-    <xsd:annotation>
-      <xsd:documentation>
-        A "hdfs-store" element specifies a HdfsStore for persistence.
-      </xsd:documentation>
-    </xsd:annotation>
-    <xsd:attribute name="name" type="xsd:string" use="required" />
-    <xsd:attribute name="namenode-url" type="xsd:string" use="optional" />
-    <xsd:attribute name="home-dir" type="xsd:string" use="optional" />
-    <xsd:attribute name="max-memory" type="xsd:string" use="optional" />
-    <xsd:attribute name="read-cache-size" type="xsd:string" use="optional" />
-    <xsd:attribute name="batch-size" type="xsd:string" use="optional" />
-    <xsd:attribute name="batch-interval" type="xsd:string" use="optional" />
-    <xsd:attribute name="dispatcher-threads" type="xsd:string" use="optional" />
-    <xsd:attribute name="buffer-persistent" type="xsd:boolean" use="optional" />
-    <xsd:attribute name="disk-store" type="xsd:string" use="optional" />
-    <xsd:attribute name="synchronous-disk-write" type="xsd:string" use="optional" />
-    <xsd:attribute name="hdfs-client-config-file" type="xsd:string" use="optional" />
-    <xsd:attribute name="purge-interval" type="xsd:string" use="optional" />
-    <xsd:attribute name="major-compaction" type="xsd:string" use="optional" />
-    <xsd:attribute name="major-compaction-interval" type="xsd:string" use="optional" />
-    <xsd:attribute name="major-compaction-threads" type="xsd:integer" use="optional" />
-    <xsd:attribute name="minor-compaction" type="xsd:string" use="optional" />
-    <xsd:attribute name="minor-compaction-threads" type="xsd:integer" use="optional" />
-    <xsd:attribute name="max-write-only-file-size" type="xsd:integer" use="optional" />
-    <xsd:attribute name="write-only-file-rollover-interval" type="xsd:string" use="optional" />    
-  </xsd:complexType>
-
   <xsd:complexType name="pdx-type">
     <xsd:annotation>
       <xsd:documentation>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/SignalledFlushObserverJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/SignalledFlushObserverJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/SignalledFlushObserverJUnitTest.java
deleted file mode 100644
index 6f69427..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/SignalledFlushObserverJUnitTest.java
+++ /dev/null
@@ -1,97 +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.hdfs.internal;
-
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.cache.hdfs.internal.FlushObserver.AsyncFlushResult;
-import com.gemstone.gemfire.cache.hdfs.internal.hoplog.AbstractHoplogOrganizer;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest
-;
-
-import junit.framework.TestCase;
-
-@Category({IntegrationTest.class})
-public class SignalledFlushObserverJUnitTest extends TestCase {
-  private AtomicInteger events;
-  private AtomicInteger delivered;
-  
-  private SignalledFlushObserver sfo;
-  
-  public void testEmpty() throws InterruptedException {
-    assertFalse(sfo.shouldDrainImmediately());
-    assertTrue(sfo.flush().waitForFlush(0, TimeUnit.NANOSECONDS));
-    assertFalse(sfo.shouldDrainImmediately());
-  }
-  
-  public void testSingle() throws InterruptedException {
-    sfo.push();
-    AsyncFlushResult result = sfo.flush();
-
-    assertTrue(sfo.shouldDrainImmediately());
-    sfo.pop(1);
-    
-    assertTrue(result.waitForFlush(0, TimeUnit.MILLISECONDS));
-    assertFalse(sfo.shouldDrainImmediately());
-  }
-
-  public void testDouble() throws InterruptedException {
-    sfo.push();
-    sfo.push();
-
-    AsyncFlushResult result = sfo.flush();
-    assertTrue(sfo.shouldDrainImmediately());
-
-    sfo.pop(1);
-    assertFalse(result.waitForFlush(0, TimeUnit.MILLISECONDS));
-
-    sfo.pop(1);
-    assertTrue(result.waitForFlush(0, TimeUnit.MILLISECONDS));
-    assertFalse(sfo.shouldDrainImmediately());
-  }
-
-  public void testTimeout() throws InterruptedException {
-    sfo.push();
-    AsyncFlushResult result = sfo.flush();
-
-    assertTrue(sfo.shouldDrainImmediately());
-    assertFalse(result.waitForFlush(100, TimeUnit.MILLISECONDS));
-    sfo.pop(1);
-    
-    assertTrue(result.waitForFlush(0, TimeUnit.MILLISECONDS));
-    assertFalse(sfo.shouldDrainImmediately());
-  }
-  
-  @Override
-  protected void setUp() {
-    events = new AtomicInteger(0);
-    delivered = new AtomicInteger(0);
-    sfo = new SignalledFlushObserver();
-    AbstractHoplogOrganizer.JUNIT_TEST_RUN = true;
-  }
-  
-  private int push() {
-    return events.incrementAndGet();
-  }
-  
-  private int pop() {
-    return delivered.incrementAndGet();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/SortedListForAsyncQueueJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/SortedListForAsyncQueueJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/SortedListForAsyncQueueJUnitTest.java
deleted file mode 100644
index fc0232f..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/SortedListForAsyncQueueJUnitTest.java
+++ /dev/null
@@ -1,564 +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.hdfs.internal;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.ConcurrentSkipListSet;
-
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.Operation;
-import com.gemstone.gemfire.cache.PartitionAttributesFactory;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionFactory;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.asyncqueue.internal.ParallelAsyncEventQueueImpl;
-import com.gemstone.gemfire.cache.hdfs.internal.HDFSBucketRegionQueue.KeyToSeqNumObject;
-import com.gemstone.gemfire.cache.hdfs.internal.HDFSBucketRegionQueue.MultiRegionSortedQueue;
-import com.gemstone.gemfire.cache.hdfs.internal.HDFSBucketRegionQueue.SortedEventQueue;
-import com.gemstone.gemfire.cache.hdfs.internal.hoplog.AbstractHoplogOrganizer;
-import com.gemstone.gemfire.cache.hdfs.internal.hoplog.HoplogConfig;
-import com.gemstone.gemfire.distributed.DistributedMember;
-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.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerHelper;
-import com.gemstone.gemfire.internal.cache.wan.GatewaySenderAttributes;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest
-;
-
-import junit.framework.TestCase;
-
-/**
- * A test class for testing whether the functionalities of sorted Aysync Queue.
- * 
- */
-@Category({IntegrationTest.class})
-public class SortedListForAsyncQueueJUnitTest extends TestCase {
-  
-  public SortedListForAsyncQueueJUnitTest() {
-    super();
-  }
-
-  private GemFireCacheImpl c;
-
-  @Override
-  public void setUp() {
-    
-    System.setProperty(HoplogConfig.ALLOW_LOCAL_HDFS_PROP, "true");
- // make it a loner
-    this.c = createCache();
-    AbstractHoplogOrganizer.JUNIT_TEST_RUN = true;
-  }
-
-  protected GemFireCacheImpl createCache() {
-    return (GemFireCacheImpl) new CacheFactory().set("mcast-port", "0").set("log-level", "warning")
-        .create();
-  }
-
-  @Override
-  public void tearDown() {
-    this.c.close();
-  }
-  
-  public void testHopQueueWithOneBucket() throws Exception {
-    this.c.close();
-    this.c = createCache();
-    PartitionAttributesFactory paf = new PartitionAttributesFactory();
-    paf.setTotalNumBuckets(1);
-    
-    RegionFactory rf1 = this.c.createRegionFactory(RegionShortcut.PARTITION);
-    PartitionedRegion r1 = (PartitionedRegion) rf1.setPartitionAttributes(paf.create()).create("r1");
-    r1.put("K9", "x1");
-    r1.put("K8", "x2");
-    // hack to get the queue. 
-    HDFSParallelGatewaySenderQueue hopqueue = getHDFSQueue(r1, this.c);
-    HDFSBucketRegionQueue hdfsBQ = (HDFSBucketRegionQueue)((PartitionedRegion)hopqueue.getRegion()).getDataStore().getLocalBucketById(0);
-    
-    EntryEventImpl ev1 = EntryEventImpl.create((LocalRegion)r1, Operation.CREATE,
-        (Object)"K1", (Object)"V1", null,
-        false, (DistributedMember)c.getMyId());
-    // put some keys with multiple updates.
-    hopqueue.put(getNewEvent("K2", "V2", r1, 0, 2) );
-    hopqueue.put(getNewEvent("K3", "V3a", r1, 0, 8) );
-    hopqueue.put(getNewEvent("K3", "V3", r1, 0, 7) );
-    hopqueue.put(getNewEvent("K1", "V1", r1, 0, 3) );
-    hopqueue.put(getNewEvent("K2", "V2a", r1, 0, 6) );
-    hopqueue.put(getNewEvent("K3", "V3b", r1, 0, 9) );
-    
-    assertTrue(" skip list size should be  6 ", getSortedEventQueue(hdfsBQ).currentSkipList.size() == 6);
-    
-    
-    // peek a key. it should be the lowesy
-    Object[] l = hopqueue.peek(1, 0).toArray();
-    
-    assertTrue("First key should be K1 but is " + ((HDFSGatewayEventImpl)l[0]).getKey(), ((HDFSGatewayEventImpl)l[0]).getKey().equals("K1"));
-    assertTrue(" Peeked skip list size should be  0 ", getSortedEventQueue(hdfsBQ).getPeeked().size() == 6);
-    assertTrue(" skip list size should be  6 ", getSortedEventQueue(hdfsBQ).currentSkipList.size() == 0);
-    
-    // try to fetch the key. it would be in peeked skip list but still available
-    Object o = hopqueue.get(r1, CacheServerHelper.serialize("K1"), 0);
-    assertTrue("First key should be K1", ((HDFSGatewayEventImpl)o).getKey().equals("K1"));
-    
-    assertTrue(" skip lists size should be  6"  , ( getSortedEventQueue(hdfsBQ).getPeeked().size() + getSortedEventQueue(hdfsBQ).currentSkipList.size() ) == 6);
-    
-    o = hopqueue.get(r1, CacheServerHelper.serialize("K2"), 0);
-    Object v = ((HDFSGatewayEventImpl)o).getDeserializedValue();
-    assertTrue(" key should K2 with value V2a but the value was " + v , ((String)v).equals("V2a"));
-    
-    o = hopqueue.get(r1, CacheServerHelper.serialize("K3"), 0);
-    v = ((HDFSGatewayEventImpl)o).getDeserializedValue();
-    assertTrue(" key should K3 with value V3b but the value was " + v , ((String)v).equals("V3b"));
-  }
-
-  protected SortedEventQueue getSortedEventQueue(HDFSBucketRegionQueue hdfsBQ) {
-    MultiRegionSortedQueue multiQueue = (MultiRegionSortedQueue)(hdfsBQ.hdfsEventQueue);
-    return multiQueue.regionToEventQueue.values().iterator().next();
-  }
-  
-  public void testPeekABatch() throws Exception {
-    this.c.close();
-    this.c = createCache();
-    PartitionAttributesFactory paf = new PartitionAttributesFactory();
-    paf.setTotalNumBuckets(1);
-    
-    RegionFactory rf1 = this.c.createRegionFactory(RegionShortcut.PARTITION);
-    PartitionedRegion r1 = (PartitionedRegion) rf1.setPartitionAttributes(paf.create()).create("r1");
-    r1.put("K9", "x1");
-    r1.put("K8", "x2");
-    // hack to get the queue. 
-    HDFSParallelGatewaySenderQueue hopqueue = getHDFSQueue(r1, this.c);
-    HDFSBucketRegionQueue hdfsBQ = (HDFSBucketRegionQueue)((PartitionedRegion)hopqueue.getRegion()).getDataStore().getLocalBucketById(0);
-    
-    
-    // put some keys with multiple updates.
-    hopqueue.put(getNewEvent("K2", "V2", r1, 0, 2) );
-    hopqueue.put(getNewEvent("K3", "V3a", r1, 0, 8) );
-    hopqueue.put(getNewEvent("K3", "V3", r1, 0, 7) );
-    hopqueue.put(getNewEvent("K1", "V1", r1, 0, 3) );
-    hopqueue.put(getNewEvent("K2", "V2a", r1, 0, 6) );
-    hopqueue.put(getNewEvent("K3", "V3b", r1, 0, 9) );
-    
-    getSortedEventQueue(hdfsBQ).rollover(true);
-    
-    hopqueue.put(getNewEvent("K1", "V12", r1, 0, 11) );
-    hopqueue.put(getNewEvent("K5", "V3a", r1, 0, 12) );
-    hopqueue.put(getNewEvent("K5", "V3b", r1, 0, 13) );
-    
-    assertTrue(" skip list size should be  3 but is " + getSortedEventQueue(hdfsBQ).currentSkipList.size(), getSortedEventQueue(hdfsBQ).currentSkipList.size() == 3);
-    assertTrue(" skip list size should be  6 but is " + getSortedEventQueue(hdfsBQ).queueOfLists.peek().size(), getSortedEventQueue(hdfsBQ).queueOfLists.peek().size() == 6);
-    
-    Object o1 = hopqueue.get(r1, CacheServerHelper.serialize("K3"), 0);;
-    Object o2 = hopqueue.get(r1, CacheServerHelper.serialize("K1"), 0);;
-    Object v1 = ((HDFSGatewayEventImpl)o1).getDeserializedValue();
-    Object v2 = ((HDFSGatewayEventImpl)o2).getDeserializedValue();
-    assertTrue(" key should K3 with value V3b but the value was " + v1 , ((String)v1).equals("V3b"));
-    assertTrue(" key should K1 with value V12 but the value was " + v2 , ((String)v2).equals("V12"));
-    
-    
-    ArrayList a = hdfsBQ.peekABatch();
-    assertTrue("First key should be K1 but is " + ((HDFSGatewayEventImpl)a.get(0)).getKey(), ((HDFSGatewayEventImpl)a.get(0)).getKey().equals("K1"));
-    assertTrue("Second key should be K2 but is " + ((HDFSGatewayEventImpl)a.get(1)).getKey(), ((HDFSGatewayEventImpl)a.get(1)).getKey().equals("K2"));
-    assertTrue("Third key should be K2 but is " + ((HDFSGatewayEventImpl)a.get(2)).getKey(), ((HDFSGatewayEventImpl)a.get(2)).getKey().equals("K2"));
-    
-    
-    assertTrue(" Peeked skip list size should be 6 ", getSortedEventQueue(hdfsBQ).getPeeked().size() == 6);
-    assertTrue(" queueOfLists size should be  2 ", getSortedEventQueue(hdfsBQ).queueOfLists.size() == 2);
-    
-    assertTrue(" skip list size should be  3 ", getSortedEventQueue(hdfsBQ).currentSkipList.size() == 3);
-    
-    o1 = hopqueue.get(r1, CacheServerHelper.serialize("K3"), 0);;
-    o2 = hopqueue.get(r1, CacheServerHelper.serialize("K1"), 0);;
-    v1 = ((HDFSGatewayEventImpl)o1).getDeserializedValue();
-    v2 = ((HDFSGatewayEventImpl)o2).getDeserializedValue();
-    assertTrue(" key should K3 with value V3b but the value was " + v1 , ((String)v1).equals("V3b"));
-    assertTrue(" key should K1 with value V12 but the value was " + v2 , ((String)v2).equals("V12"));
-    
-    
-    java.util.Iterator<KeyToSeqNumObject> iter1 = getSortedEventQueue(hdfsBQ).getPeeked().iterator();
-    assertTrue("key in peeked list should be 3 ", iter1.next().getSeqNum() == 3);
-    assertTrue("key in peeked list should be 6 ", iter1.next().getSeqNum() == 6);
-    assertTrue("key in peeked list should be 2 ", iter1.next().getSeqNum() == 2);
-    assertTrue("key in peeked list should be 9 ", iter1.next().getSeqNum() == 9);
-    assertTrue("key in peeked list should be 8 ", iter1.next().getSeqNum() == 8);
-    assertTrue("key in peeked list should be 7 ", iter1.next().getSeqNum() == 7);
-    assertTrue(" Peeked list should not have any more elements. ", iter1.hasNext() == false);
-    
-    
-    java.util.Iterator<KeyToSeqNumObject> iter2 = getSortedEventQueue(hdfsBQ).currentSkipList.iterator();
-    assertTrue("key in peeked list should be 11", iter2.next().getSeqNum() == 11);
-    assertTrue("key in peeked list should be 13", iter2.next().getSeqNum() == 13);
-    assertTrue("key in peeked list should be 12 ", iter2.next().getSeqNum() == 12);
-    
-    iter2 = getSortedEventQueue(hdfsBQ).currentSkipList.iterator();
-    HashSet<Long> hs = new HashSet<Long>();
-    hs.add((long) 11);
-    hs.add((long) 13);
-    hs.add((long) 12);
-    hs.add((long) 3);
-    hs.add((long) 6);
-    hs.add((long) 2);
-    hs.add((long) 9);
-    hs.add((long) 8);
-    hs.add((long) 7);
-    
-    hdfsBQ.hdfsEventQueue.handleRemainingElements(hs);
-    
-    ArrayList a1 = hdfsBQ.peekABatch();
-    o1 = hopqueue.get(r1, CacheServerHelper.serialize("K3"), 0);;
-    o2 = hopqueue.get(r1, CacheServerHelper.serialize("K1"), 0);;
-    v2 = ((HDFSGatewayEventImpl)o2).getDeserializedValue();
-    assertTrue(" key should K3 should not have been found ",  o1 ==null);
-    assertTrue(" key should K1 with value V12 but the value was " + v2 , ((String)v2).equals("V12"));
-    
-    assertTrue("First key should be K1 but is " + ((HDFSGatewayEventImpl)a1.get(0)).getKey(), ((HDFSGatewayEventImpl)a1.get(0)).getKey().equals("K1"));
-    assertTrue("Second key should be K5 but is " + ((HDFSGatewayEventImpl)a1.get(1)).getKey(), ((HDFSGatewayEventImpl)a1.get(1)).getKey().equals("K5"));
-    assertTrue("Third key should be K5 but is " + ((HDFSGatewayEventImpl)a1.get(2)).getKey(), ((HDFSGatewayEventImpl)a1.get(2)).getKey().equals("K5"));
-    
-    assertTrue(" Peeked skip list size should be  3 ", getSortedEventQueue(hdfsBQ).getPeeked().size() == 3);
-    assertTrue(" skip list size should be  0 but is " + getSortedEventQueue(hdfsBQ).currentSkipList.size(), getSortedEventQueue(hdfsBQ).currentSkipList.size() == 0);
-    assertTrue(" skip list size should be  3 but is " + getSortedEventQueue(hdfsBQ).queueOfLists.peek().size(), getSortedEventQueue(hdfsBQ).queueOfLists.peek().size() == 3);
-    assertTrue(" skip list size should be  2 but is " + getSortedEventQueue(hdfsBQ).queueOfLists.size(), getSortedEventQueue(hdfsBQ).queueOfLists.size() == 2);
-    
-  }
-  
-  private HDFSGatewayEventImpl getNewEvent(Object key, Object value, Region r1, int bid, int tailKey) throws Exception {
-    EntryEventImpl ev1 = EntryEventImpl.create((LocalRegion)r1, Operation.CREATE,
-        key, value, null,
-        false, (DistributedMember)c.getMyId());
-    ev1.setEventId(new EventID(this.c.getDistributedSystem()));
-    HDFSGatewayEventImpl event = null;
-    event = new HDFSGatewayEventImpl(EnumListenerEvent.AFTER_CREATE, ev1, null , true, bid);
-    event.setShadowKey((long)tailKey);
-    return event;
-  }
-  
-  /**
-   * Creates the HDFS Queue instance for a region (this skips the creation of 
-   * event processor)
-   */
-  private HDFSParallelGatewaySenderQueue getHDFSQueue(Region region, Cache c) {
-    GatewaySenderAttributes gattrs = new GatewaySenderAttributes();
-    gattrs.isHDFSQueue = true;
-    gattrs.id = "SortedListForAsyncQueueJUnitTest_test";
-    ParallelAsyncEventQueueImpl gatewaySender = new ParallelAsyncEventQueueImpl(c, gattrs);
-    HashSet<Region> set = new HashSet<Region>();
-    set.add(region);
-    HDFSParallelGatewaySenderQueue queue = new HDFSParallelGatewaySenderQueue(gatewaySender, set, 0, 1);
-    queue.start();
-    return queue;
-  }
-  
- // A test for testing whether the KeyToSeqNumObject compare function is in order.
-  public void testIfTheKeyToSeqNumIsKeptSortedWithoutConflation() throws Exception {
-    byte[] k1 = new byte[] { 1};
-    byte[] k2 = new byte[] { 2};
-    byte[] k3 = new byte[] { 3};
-    byte[] k4 = new byte[] { 4};
-    
-    KeyToSeqNumObject keyToSeq1 = new KeyToSeqNumObject(k1, new Long(2));
-    KeyToSeqNumObject keyToSeq2 = new KeyToSeqNumObject(k1, new Long(5));
-    KeyToSeqNumObject keyToSeq3 = new KeyToSeqNumObject(k1, new Long(8));
-    KeyToSeqNumObject keyToSeq4 = new KeyToSeqNumObject(k2, new Long(3));
-    KeyToSeqNumObject keyToSeq5 = new KeyToSeqNumObject(k2, new Long(7));
-    
-    ConcurrentSkipListSet<KeyToSeqNumObject> list = new ConcurrentSkipListSet<HDFSBucketRegionQueue.KeyToSeqNumObject>();
-    list.add(keyToSeq4);
-    list.add(keyToSeq3);
-    list.add(keyToSeq5);
-    list.add(keyToSeq1);
-    list.add(keyToSeq2);
-    list.add(keyToSeq5);
-    KeyToSeqNumObject k = list.pollFirst();
-    this.c.getLoggerI18n().fine(" KeyToSeqNumObject  byte: " + k.getRegionkey()[0] + " seq num: " + k.getSeqNum());
-    assertTrue ("Order of elements in Concurrent list is not correct ", k.equals(keyToSeq3));
-    list.remove(k);
-    
-    k = list.pollFirst();
-    this.c.getLoggerI18n().fine(" KeyToSeqNumObject  byte: " + k.getRegionkey()[0] + " seq num: " + k.getSeqNum());
-    assertTrue ("Order of elements in Concurrent list is not correct ", k.equals(keyToSeq2));
-    list.remove(k);
-    
-    k = list.pollFirst();
-    this.c.getLoggerI18n().fine(" KeyToSeqNumObject  byte: " + k.getRegionkey()[0] + " seq num: " + k.getSeqNum());
-    assertTrue ("Order of elements in Concurrent list is not correct ", k.equals(keyToSeq1));
-    list.remove(k);
-    
-    list.add(keyToSeq4);
-    list.add(keyToSeq3);
-    list.add(keyToSeq5);
-    list.add(keyToSeq1);
-    k = list.pollFirst();
-    this.c.getLoggerI18n().fine(" KeyToSeqNumObject  byte: " + k.getRegionkey()[0] + " seq num: " + k.getSeqNum());
-    assertTrue ("Order of elements in Concurrent list is not correct ", k.equals(keyToSeq3));
-    list.remove(k);
-    
-    k = list.pollFirst();
-    this.c.getLoggerI18n().fine(" KeyToSeqNumObject  byte: " + k.getRegionkey()[0] + " seq num: " + k.getSeqNum());
-    assertTrue ("Order of elements in Concurrent list is not correct ", k.equals(keyToSeq1));
-    list.remove(k);
-    
-    k = list.pollFirst();
-    this.c.getLoggerI18n().fine(" KeyToSeqNumObject  byte: " + k.getRegionkey()[0] + " seq num: " + k.getSeqNum());
-    assertTrue ("Order of elements in Concurrent list is not correct ", k.equals(keyToSeq5));
-    list.remove(k);
-    
-    k = list.pollFirst();
-    this.c.getLoggerI18n().fine(" KeyToSeqNumObject  byte: " + k.getRegionkey()[0] + " seq num: " + k.getSeqNum());
-    assertTrue ("Order of elements in Concurrent list is not correct ", k.equals(keyToSeq4));
-    
-    list.remove(k);
-  }
-  
-  public void testSingleGet() throws Exception {
-    checkQueueGet("K1", new KeyValue("K1", "V1"), "K1-V1");
-  }
-  
-  public void testMissingGet() throws Exception {
-    checkQueueGet("K1", null, 
-        "K0-V0",
-        "K2-V2");
-  }
-
-  public void testMultipleGet() throws Exception {
-    checkQueueGet("K1", new KeyValue("K1", "V1"), 
-        "K0-V0",
-        "K1-V1",
-        "K2-V2");
-  }
-
-  public void testDuplicateGet() throws Exception {
-    checkQueueGet("K1", new KeyValue("K1", "V1.4"), 
-        "K0-V0",
-        "K1-V1.0",
-        "K1-V1.1",
-        "K1-V1.2",
-        "K1-V1.3",
-        "K1-V1.4",
-        "K2-V2");
-  }
-
-  public void testEmptyIterator() throws Exception {
-    checkQueueIteration(Collections.<KeyValue>emptyList());
-  }
-  
-  public void testSingleIterator() throws Exception {
-    checkQueueIteration(getExpected(), 
-        "K0-V0",
-        "K1-V1",
-        "K2-V2",
-        "K3-V3",
-        "K4-V4",
-        "K5-V5",
-        "K6-V6",
-        "K7-V7",
-        "K8-V8",
-        "K9-V9"
-        );
-  }
-
-  public void testMultipleIterator() throws Exception {
-    checkQueueIteration(getExpected(), 
-        "K0-V0",
-        "K1-V1",
-        "K2-V2",
-        "roll",
-        "K3-V3",
-        "K4-V4",
-        "K5-V5",
-        "K6-V6",
-        "roll",
-        "K7-V7",
-        "K8-V8",
-        "K9-V9"
-        );
-  }
-
-  public void testMixedUpIterator() throws Exception {
-    checkQueueIteration(getExpected(), 
-        "K0-V0",
-        "K5-V5",
-        "K9-V9",
-        "roll",
-        "K3-V3",
-        "K2-V2",
-        "K6-V6",
-        "roll",
-        "K4-V4",
-        "K7-V7",
-        "K8-V8",
-        "K1-V1"
-        );
-  }
-
-  public void testMixedUpIterator2() throws Exception {
-    List<KeyValue> expected = new ArrayList<KeyValue>();
-    expected.add(new KeyValue("K0", "V0"));
-    expected.add(new KeyValue("K1", "V1.2"));
-    expected.add(new KeyValue("K2", "V2.1"));
-    expected.add(new KeyValue("K3", "V3.1"));
-    expected.add(new KeyValue("K4", "V4.2"));
-    expected.add(new KeyValue("K5", "V5.2"));
-    expected.add(new KeyValue("K6", "V6"));
-    expected.add(new KeyValue("K7", "V7"));
-    expected.add(new KeyValue("K8", "V8"));
-    expected.add(new KeyValue("K9", "V9"));
-    
-    checkQueueIteration(expected, 
-        "K1-V1.0",
-        "K2-V2.0",
-        "K3-V3.0",
-        "K4-V4.0",
-        "roll",
-        "K2-V2.1",
-        "K4-V4.1",
-        "K6-V6",
-        "K8-V8",
-        "roll",
-        "K1-V1.1",
-        "K3-V3.1",
-        "K5-V5.0",
-        "K7-V7",
-        "K9-V9",
-        "roll",
-        "K0-V0",
-        "K1-V1.2",
-        "K4-V4.2",
-        "K5-V5.1",
-        "K5-V5.2"
-        );
-  }
-
-  private List<KeyValue> getExpected() {
-    List<KeyValue> expected = new ArrayList<KeyValue>();
-    expected.add(new KeyValue("K0", "V0"));
-    expected.add(new KeyValue("K1", "V1"));
-    expected.add(new KeyValue("K2", "V2"));
-    expected.add(new KeyValue("K3", "V3"));
-    expected.add(new KeyValue("K4", "V4"));
-    expected.add(new KeyValue("K5", "V5"));
-    expected.add(new KeyValue("K6", "V6"));
-    expected.add(new KeyValue("K7", "V7"));
-    expected.add(new KeyValue("K8", "V8"));
-    expected.add(new KeyValue("K9", "V9"));
-    
-    return expected;
-  }
-  
-  private void checkQueueGet(String key, KeyValue expected, String... entries) throws Exception {
-    PartitionAttributesFactory paf = new PartitionAttributesFactory();
-    paf.setTotalNumBuckets(1);
-    
-    RegionFactory rf1 = this.c.createRegionFactory(RegionShortcut.PARTITION);
-    PartitionedRegion r1 = (PartitionedRegion) rf1.setPartitionAttributes(paf.create()).create("r1");
-
-    // create the buckets
-    r1.put("blah", "blah");
-
-    // hack to get the queue. 
-    HDFSParallelGatewaySenderQueue hopqueue = getHDFSQueue(r1, this.c);
-    HDFSBucketRegionQueue brq = (HDFSBucketRegionQueue)((PartitionedRegion)hopqueue.getRegion()).getDataStore().getLocalBucketById(0);
-
-    
-    int seq = 0;
-    for (String s : entries) {
-      if (s.equals("roll")) {
-        brq.rolloverSkipList();
-      } else {
-        String[] kv = s.split("-");
-        hopqueue.put(getNewEvent(kv[0], kv[1], r1, 0, seq++));
-      }
-    }
-
-    byte[] bkey = EntryEventImpl.serialize(key);
-    HDFSGatewayEventImpl evt = hopqueue.get(r1, bkey, 0);
-    if (expected == null) {
-      assertNull(evt);
-      
-    } else {
-      assertEquals(expected.key, evt.getKey());
-      assertEquals(expected.value, evt.getDeserializedValue());
-    }
-  }
-  
-  private void checkQueueIteration(List<KeyValue> expected, String... entries) throws Exception {
-    PartitionAttributesFactory paf = new PartitionAttributesFactory();
-    paf.setTotalNumBuckets(1);
-    
-    RegionFactory rf1 = this.c.createRegionFactory(RegionShortcut.PARTITION);
-    Region r1 = rf1.setPartitionAttributes(paf.create()).create("r1");
-
-    // create the buckets
-    r1.put("blah", "blah");
-
-    HDFSParallelGatewaySenderQueue hopqueue = getHDFSQueue(r1, this.c);
-    HDFSBucketRegionQueue brq = (HDFSBucketRegionQueue)((PartitionedRegion)hopqueue.getRegion()).getDataStore().getLocalBucketById(0);
-    
-    int seq = 0;
-    for (String s : entries) {
-      if (s.equals("roll")) {
-        brq.rolloverSkipList();
-      } else {
-        String[] kv = s.split("-");
-        hopqueue.put(getNewEvent(kv[0], kv[1], r1, 0, seq++));
-        getSortedEventQueue(brq).rollover(true);
-      }
-    }
-    
-    Iterator<HDFSGatewayEventImpl> iter = brq.iterator(r1);
-    List<KeyValue> actual = new ArrayList<KeyValue>();
-    while (iter.hasNext()) {
-      HDFSGatewayEventImpl evt = iter.next();
-      actual.add(new KeyValue((String) evt.getKey(), (String) evt.getDeserializedValue()));
-    }
-    
-    assertEquals(expected, actual);
-  }
-  
-  public static class KeyValue {
-    public final String key;
-    public final String value;
-    
-    public KeyValue(String key, String value) {
-      this.key = key;
-      this.value = value;
-    }
-    
-    @Override
-    public boolean equals(Object o) {
-      if (o == null)
-        return false;
-
-      KeyValue obj = (KeyValue) o;
-      return key.equals(obj.key) && value.equals(obj.value);
-    }
-    
-    @Override
-    public String toString() {
-      return key + "=" + value;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/test/java/com/gemstone/gemfire/cache30/Bug38741DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/Bug38741DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/Bug38741DUnitTest.java
index f1b9746..7e4acbf 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/Bug38741DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/Bug38741DUnitTest.java
@@ -308,7 +308,7 @@ public class Bug38741DUnitTest extends ClientServerTestCase {
             BucketRegion br = (BucketRegion) r;
             try {
               KeyInfo keyInfo = new KeyInfo(k1, null, bucketId);
-              RawValue rv = br.getSerialized(keyInfo, false, false, null, null, false, false);
+              RawValue rv = br.getSerialized(keyInfo, false, false, null, null, false);
               Object val = rv.getRawValue();
               assertTrue(val instanceof CachedDeserializable);
               CachedDeserializable cd = (CachedDeserializable)val;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueJUnitTest.java
index a7daf98..b2399fd 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueJUnitTest.java
@@ -67,7 +67,7 @@ public class ParallelGatewaySenderQueueJUnitTest {
     PartitionedRegionDataStore dataStore = mock(PartitionedRegionDataStore.class);
     when(mockMetaRegion.getDataStore()).thenReturn(dataStore);
     when(dataStore.getSizeOfLocalPrimaryBuckets()).thenReturn(3); 
-    when(metaRegionFactory.newMetataRegion(any(), any(), any(), any(), anyBoolean())).thenReturn(mockMetaRegion);
+    when(metaRegionFactory.newMetataRegion(any(), any(), any(), any())).thenReturn(mockMetaRegion);
     when(cache.createVMRegion(any(), any(), any())).thenReturn(mockMetaRegion);
     
     queue.addShadowPartitionedRegionForUserPR(mockPR("region1"));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46535f28/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/domain/CacheElementJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/domain/CacheElementJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/domain/CacheElementJUnitTest.java
index dca5d0b..57d1c7e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/domain/CacheElementJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/configuration/domain/CacheElementJUnitTest.java
@@ -120,7 +120,6 @@ public class CacheElementJUnitTest {
     assertEntry("cache-server", order++, entries.next());
     assertEntry("pool", order++, entries.next());
     assertEntry("disk-store", order++, entries.next());
-    assertEntry("hdfs-store", order++, entries.next());
     assertEntry("pdx", order++, entries.next());
     assertEntry("region-attributes", order++, entries.next());
     assertEntry("jndi-bindings", order++, entries.next());