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/06/18 21:05:47 UTC

[01/16] incubator-geode git commit: GEDOE-93: Entry count stats are incorrect with PR with entry eviction and async disk

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-1565 ca5b45cc8 -> db739b221


GEDOE-93: Entry count stats are incorrect with PR with entry eviction and async disk

 * Refactored disk status update logic to do it at one place
 * Lazily updating stats for disk async case to avoid complex
   logic to undo stats in case if there is another operation
   before the data is flushed to disk
 * Fixed a wan test and wrapped the asserts inside awaitility


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

Branch: refs/heads/feature/GEODE-1565
Commit: 8a132221fcfe6d2e2e8ba97628307c5fd993c047
Parents: 3dd2efd
Author: Sai Boorlagadda <sb...@pivotal.io>
Authored: Mon May 23 13:10:47 2016 -0700
Committer: Sai Boorlagadda <sb...@pivotal.io>
Committed: Thu Jun 16 11:46:34 2016 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/DiskEntry.java       | 88 +++++++++++---------
 .../gemfire/internal/cache/LocalRegion.java     |  4 -
 .../cache/PartitionedRegionStatsJUnitTest.java  | 26 ++----
 ...llelGatewaySenderQueueOverflowDUnitTest.java | 42 ++++++----
 4 files changed, 76 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a132221/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 5da0d9a..698e3bd 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
@@ -897,30 +897,31 @@ public interface DiskEntry extends RegionEntry {
           did.setUserBits(re.getUserBits());
           oldValueLength = did.getValueLength();
           did.setValueLength(re.getValueLength());
-          // The following undo and then do fixes bug 41849
-          // First, undo the stats done for the previous recovered value
-          if (oldKeyId < 0) {
-            dr.incNumOverflowOnDisk(-1L);
-            dr.incNumOverflowBytesOnDisk(-oldValueLength);
-            incrementBucketStats(region, 0/*InVM*/, -1/*OnDisk*/, -oldValueLength);
-          } else {
-            dr.incNumEntriesInVM(-1L);
-            incrementBucketStats(region, -1/*InVM*/, 0/*OnDisk*/, 0);
-          }
-          // Second, do the stats done for the current recovered value
+          
           if (re.getRecoveredKeyId() < 0) {
             if (!entry.isValueNull()) {
               entry.handleValueOverflow(region);
               entry.setValueWithContext(region, null); // fixes bug 41119
             }
-            dr.incNumOverflowOnDisk(1L);
-            dr.incNumOverflowBytesOnDisk(did.getValueLength());
-            incrementBucketStats(region, 0/*InVM*/, 1/*OnDisk*/,
-                                 did.getValueLength());
           } else {
             entry.setValueWithContext(region, entry.prepareValueForCache(region, re.getValue(), false));
-            dr.incNumEntriesInVM(1L);
-            incrementBucketStats(region, 1/*InVM*/, 0/*OnDisk*/, 0);
+          }
+          
+          if (re.getRecoveredKeyId() < 0) {
+            if(oldKeyId >= 0) {
+              dr.incNumEntriesInVM(-1L);
+              dr.incNumOverflowOnDisk(1L);
+              dr.incNumOverflowBytesOnDisk(did.getValueLength());
+              incrementBucketStats(region, -1/*InVM*/, 1/*OnDisk*/,
+                                   did.getValueLength());
+            }
+          } else {
+            if(oldKeyId < 0) {
+              dr.incNumEntriesInVM(1L);
+              dr.incNumOverflowOnDisk(-1L);
+              dr.incNumOverflowBytesOnDisk(-oldValueLength);
+              incrementBucketStats(region, 1/*InVM*/, -1/*OnDisk*/, -oldValueLength);
+            }
           }
         }
         else {
@@ -993,24 +994,33 @@ public interface DiskEntry extends RegionEntry {
             dr.incNumEntriesInVM(1L);
             incrementBucketStats(region, 1/*InVM*/, 0/*OnDisk*/, 0);
           }
-        }
-        if (entry instanceof LRUEntry) {
-          LRUEntry le = (LRUEntry)entry;
-          boolean wasEvicted = le.testEvicted();
-          le.unsetEvicted();
-          if (!Token.isRemovedFromDisk(newValue)) {
-            if (oldValue == null
-                // added null check for bug 41759
-                || wasEvicted && did != null && did.isPendingAsync()) {
-              // Note we do not append this entry because that will be
-              // done by lruEntryUpdate
+          
+          if(newValue == Token.TOMBSTONE) {
+            if (oldValue == null) {
+              dr.incNumOverflowOnDisk(-1L);
+              dr.incNumOverflowBytesOnDisk(-oldValueLength);
+              incrementBucketStats(region, 0/*InVM*/, -1/*OnDisk*/, -oldValueLength);
+            } else {
+              dr.incNumEntriesInVM(-1L);
+              incrementBucketStats(region, -1/*InVM*/, 0/*OnDisk*/, 0);
+            }
+          } else {
+            if (oldValue == null) {
               dr.incNumEntriesInVM(1L);
               dr.incNumOverflowOnDisk(-1L);
               dr.incNumOverflowBytesOnDisk(-oldValueLength);
               incrementBucketStats(region, 1/*InVM*/, -1/*OnDisk*/, -oldValueLength);
+            } else if(oldValue == Token.TOMBSTONE) {
+              dr.incNumEntriesInVM(1L);
+              incrementBucketStats(region, 1/*InVM*/, 0/*OnDisk*/, 0/*overflowBytesOnDisk*/);
             }
           }
         }
+        if (entry instanceof LRUEntry) {
+          LRUEntry le = (LRUEntry)entry;
+          le.unsetEvicted();          
+        }
+
       }
       } finally {
         if (syncObj == did) {
@@ -1158,11 +1168,6 @@ public interface DiskEntry extends RegionEntry {
                 // Seems like we could end up adding it to the queue multiple times.
                 did.setPendingAsync(false);
               }
-              // since it was evicted fix the stats here
-              dr.incNumEntriesInVM(1L);
-              dr.incNumOverflowOnDisk(-1L);
-              // no need to dec overflowBytesOnDisk because it was not inced in this case.
-              incrementBucketStats(region, 1/*InVM*/, -1/*OnDisk*/, 0);
             }
             lruEntryFaultIn((LRUEntry) entry, region);
             lruFaultedIn = true;
@@ -1445,7 +1450,6 @@ public interface DiskEntry extends RegionEntry {
             // and now we are faulting it out
           }
         }
-
         boolean movedValueToDisk = false; // added for bug 41849
         
         // If async then if it does not need to be written (because it already was)
@@ -1465,10 +1469,12 @@ public interface DiskEntry extends RegionEntry {
         if (movedValueToDisk) {
           valueLength = getValueLength(did);
         }
-        dr.incNumEntriesInVM(-1L);
-        dr.incNumOverflowOnDisk(1L);
-        dr.incNumOverflowBytesOnDisk(valueLength);
-        incrementBucketStats(region, -1/*InVM*/, 1/*OnDisk*/, valueLength);
+        if(dr.isSync() || movedValueToDisk) {
+          dr.incNumEntriesInVM(-1L);
+          dr.incNumOverflowOnDisk(1L);
+          dr.incNumOverflowBytesOnDisk(valueLength);
+          incrementBucketStats(region, -1/*InVM*/, 1/*OnDisk*/, valueLength);
+        }
       }
       } finally {
         dr.releaseReadLock();
@@ -1650,10 +1656,10 @@ public interface DiskEntry extends RegionEntry {
                   && ((LRUEntry)entry).testEvicted()) {
                 // Moved this here to fix bug 40116.
                 region.updateSizeOnEvict(entry.getKey(), entryValSize);
-                // note the old size was already accounted for
-                // onDisk was already inced so just do the valueLength here
+                dr.incNumEntriesInVM(-1);
+                dr.incNumOverflowOnDisk(1L);
                 dr.incNumOverflowBytesOnDisk(did.getValueLength());
-                incrementBucketStats(region, 0/*InVM*/, 0/*OnDisk*/,
+                incrementBucketStats(region, -1/*InVM*/, 1/*OnDisk*/,
                                      did.getValueLength());
                 entry.handleValueOverflow(region);
                 entry.setValueWithContext(region,null);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a132221/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 e5897cc..205f38f 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
@@ -3280,10 +3280,6 @@ public class LocalRegion extends AbstractRegion
     //Fix for 45204 - don't include the tombstones in
     //any of our entry count stats.
     this.cachePerfStats.incEntryCount(-delta);
-    if(getDiskRegion() != null) {
-      getDiskRegion().incNumEntriesInVM(-delta);
-    }
-    DiskEntry.Helper.incrementBucketStats(this, -delta/*InVM*/, 0/*OnDisk*/, 0);
   }
   
   public int getTombstoneCount() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a132221/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionStatsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionStatsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionStatsJUnitTest.java
index 1a3277c..82e3489 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionStatsJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionStatsJUnitTest.java
@@ -475,30 +475,14 @@ public class PartitionedRegionStatsJUnitTest
     assertEquals(singleEntryMemSize * entriesInMem, stats.getLong("dataStoreBytesInUse"));
     assertEquals(numEntries , stats.getInt("dataStoreEntryCount"));
     assertEquals((numEntries - entriesInMem) * entryOverflowSize, diskStats.getNumOverflowBytesOnDisk());
-    //Disabled for GEODE-93. numEntriesInVM and numOVerflowOnDisk are incorrect
-//    assertIndexDetailsEquals(entriesInMem , diskStats.getNumEntriesInVM());
-//    assertIndexDetailsEquals((numEntries - entriesInMem) , diskStats.getNumOverflowOnDisk());
-      assertEquals(stats.getLong("dataStoreBytesInUse"), getMemBytes(pr));
-      assertEquals(diskStats.getNumOverflowBytesOnDisk(), getDiskBytes(pr));
-    }
-
-  private int countEntriesInMem(PartitionedRegion pr) {
-    int entriesInMem = 0;
-    for(BucketRegion br : pr.getDataStore().getAllLocalBucketRegions()) {
-      for(RegionEntry entry : br.entries.regionEntries()) {
-        if(entry._getValue() != null && !Token.isRemoved(entry._getValue())) {
-          System.out.println("Still in memory " + entry.getKey());
-          entriesInMem++;
-        }
-      }
-    }
-    
-    System.out.println("EntriesInMem = " + entriesInMem);
-    return entriesInMem;
+    assertEquals(entriesInMem , diskStats.getNumEntriesInVM());
+    assertEquals((numEntries - entriesInMem) , diskStats.getNumOverflowOnDisk());
+    assertEquals(stats.getLong("dataStoreBytesInUse"), getMemBytes(pr));
+    assertEquals(diskStats.getNumOverflowBytesOnDisk(), getDiskBytes(pr));
   }
 
   private Object getDiskBytes(PartitionedRegion pr) {
-Set<BucketRegion> brs = pr.getDataStore().getAllLocalBucketRegions();
+    Set<BucketRegion> brs = pr.getDataStore().getAllLocalBucketRegions();
     
     long bytes = 0;
     for(Iterator<BucketRegion> itr = brs.iterator(); itr.hasNext(); ) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8a132221/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueOverflowDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueOverflowDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueOverflowDUnitTest.java
index e2fe10b..44308f8 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueOverflowDUnitTest.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueueOverflowDUnitTest.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.*;
 import java.io.File;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
 import org.junit.Ignore;
 import org.junit.Test;
@@ -49,6 +50,7 @@ import com.gemstone.gemfire.test.dunit.LogWriterUtils;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.jayway.awaitility.Awaitility;
 
 /**
  * DUnit for ParallelSenderQueue overflow operations.
@@ -98,26 +100,30 @@ public class ParallelGatewaySenderQueueOverflowDUnitTest extends WANTestBase {
     int numEventPuts = 50;
     vm4.invoke(() -> WANTestBase.doHeavyPuts( getTestMethodName(), numEventPuts ));
     
-    long numOvVm4 = (Long) vm4.invoke(() -> WANTestBase.getNumberOfEntriesOverflownToDisk( "ln" ));
-    long numOvVm5 = (Long) vm5.invoke(() -> WANTestBase.getNumberOfEntriesOverflownToDisk( "ln" ));
-    long numOvVm6 = (Long) vm6.invoke(() -> WANTestBase.getNumberOfEntriesOverflownToDisk( "ln" ));
-    long numOvVm7 = (Long) vm7.invoke(() -> WANTestBase.getNumberOfEntriesOverflownToDisk( "ln" ));
-    
-    long numMemVm4 = (Long) vm4.invoke(() -> WANTestBase.getNumberOfEntriesInVM( "ln" ));
-    long numMemVm5 = (Long) vm5.invoke(() -> WANTestBase.getNumberOfEntriesInVM( "ln" ));
-    long numMemVm6 = (Long) vm6.invoke(() -> WANTestBase.getNumberOfEntriesInVM( "ln" ));
-    long numMemVm7 = (Long) vm7.invoke(() -> WANTestBase.getNumberOfEntriesInVM( "ln" ));
     
-    LogWriterUtils.getLogWriter().info("Entries overflown to disk: " + numOvVm4 + "," + numOvVm5 + "," + numOvVm6 + "," + numOvVm7);
-    LogWriterUtils.getLogWriter().info("Entries in VM: " + numMemVm4 + "," + numMemVm5 + "," + numMemVm6 + "," + numMemVm7);
-    
-    long totalOverflown = numOvVm4 + numOvVm5 + numOvVm6 + numOvVm7; 
     //considering a memory limit of 40 MB, maximum of 40 events can be in memory. Rest should be on disk.
-    assertTrue("Total number of entries overflown to disk should be at least greater than 55", (totalOverflown > 55));
-    
-    long totalInMemory = numMemVm4 + numMemVm5 + numMemVm6 + numMemVm7;
-    //expected is twice the number of events put due to redundancy level of 1  
-    assertEquals("Total number of entries on disk and in VM is incorrect", (numEventPuts*2), (totalOverflown + totalInMemory));
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(()-> 
+    {
+      long numOvVm4 = (Long) vm4.invoke(() -> WANTestBase.getNumberOfEntriesOverflownToDisk( "ln" ));
+      long numOvVm5 = (Long) vm5.invoke(() -> WANTestBase.getNumberOfEntriesOverflownToDisk( "ln" ));
+      long numOvVm6 = (Long) vm6.invoke(() -> WANTestBase.getNumberOfEntriesOverflownToDisk( "ln" ));
+      long numOvVm7 = (Long) vm7.invoke(() -> WANTestBase.getNumberOfEntriesOverflownToDisk( "ln" ));
+      
+      long numMemVm4 = (Long) vm4.invoke(() -> WANTestBase.getNumberOfEntriesInVM( "ln" ));
+      long numMemVm5 = (Long) vm5.invoke(() -> WANTestBase.getNumberOfEntriesInVM( "ln" ));
+      long numMemVm6 = (Long) vm6.invoke(() -> WANTestBase.getNumberOfEntriesInVM( "ln" ));
+      long numMemVm7 = (Long) vm7.invoke(() -> WANTestBase.getNumberOfEntriesInVM( "ln" ));
+      
+      LogWriterUtils.getLogWriter().info("Entries overflown to disk: " + numOvVm4 + "," + numOvVm5 + "," + numOvVm6 + "," + numOvVm7);
+      LogWriterUtils.getLogWriter().info("Entries in VM: " + numMemVm4 + "," + numMemVm5 + "," + numMemVm6 + "," + numMemVm7);
+      long totalOverflown = numOvVm4 + numOvVm5 + numOvVm6 + numOvVm7; 
+      assertTrue("Total number of entries overflown to disk should be at least greater than 55", (totalOverflown > 55));
+
+      long totalInMemory = numMemVm4 + numMemVm5 + numMemVm6 + numMemVm7;
+      //expected is twice the number of events put due to redundancy level of 1  
+      assertEquals("Total number of entries on disk and in VM is incorrect", (numEventPuts*2), (totalOverflown + totalInMemory));
+      
+    });
     
     vm4.invoke(() -> WANTestBase.resumeSender( "ln" ));
     vm5.invoke(() -> WANTestBase.resumeSender( "ln" ));


[12/16] incubator-geode git commit: GEODE-1563: apply FlakyTest category to testExecuteFunctionOnGroups

Posted by kl...@apache.org.
GEODE-1563: apply FlakyTest category to testExecuteFunctionOnGroups


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

Branch: refs/heads/feature/GEODE-1565
Commit: 6ffef33ed7c256e33c6871dbd52826b07a741e5c
Parents: 947628a
Author: Kirk Lund <kl...@apache.org>
Authored: Fri Jun 17 11:46:04 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Jun 17 13:19:47 2016 -0700

----------------------------------------------------------------------
 .../internal/cli/commands/FunctionCommandsDUnitTest.java          | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6ffef33e/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java
index 735de4f..478f9cd 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommandsDUnitTest.java
@@ -32,6 +32,8 @@ import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.test.dunit.*;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
+
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -405,6 +407,7 @@ public class FunctionCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Category(FlakyTest.class) // GEODE-1563: JMX RMI (java.rmi.NoSuchObjectException: no such object in table)
   @Test
   public void testExecuteFunctionOnGroups() {
     Properties localProps = new Properties();


[04/16] incubator-geode git commit: GEODE-11: Added LuceneQueryException to search signature

Posted by kl...@apache.org.
GEODE-11: Added LuceneQueryException to search signature

Search will now throw a LuceneQueryException if a query cannot be parsed.

This closes #163


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

Branch: refs/heads/feature/GEODE-1565
Commit: 2fe3a4b2cb0399f71b15c99824807d822f5e0ab0
Parents: 786c862
Author: Aparna Dharmakkan <ad...@pivotal.io>
Authored: Thu Jun 16 14:12:42 2016 -0700
Committer: nabarun <nn...@pivotal.io>
Committed: Thu Jun 16 16:14:35 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneQuery.java       |  2 +-
 .../cache/lucene/LuceneQueryException.java      |  4 +++
 .../cache/lucene/internal/LuceneQueryImpl.java  | 17 +++++++---
 ...IndexCreationPersistenceIntegrationTest.java | 17 +++++++---
 .../lucene/LuceneQueriesIntegrationTest.java    | 34 ++++++--------------
 .../LuceneQueryImplIntegrationTest.java         |  2 +-
 .../distributed/LuceneFunctionJUnitTest.java    |  3 +-
 .../cache/lucene/test/LuceneTestUtilities.java  |  5 +--
 8 files changed, 46 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
index 2de9c0b..93426b9 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
@@ -29,7 +29,7 @@ public interface LuceneQuery<K, V> {
   /**
    * Execute the search and get results. 
    */
-  public LuceneQueryResults<K, V> search();
+  public LuceneQueryResults<K, V> search() throws LuceneQueryException;
   
   /**
    * Get page size setting of current query. 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java
index 683b799..fb03b4a 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java
@@ -33,4 +33,8 @@ public class LuceneQueryException extends GemFireCheckedException {
   public LuceneQueryException(final String message, final Throwable cause) {
     super(message, cause);
   }
+
+  public LuceneQueryException(final Throwable cause) {
+    super(cause);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
index c6b8878..92f1957 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
@@ -20,9 +20,11 @@
 package com.gemstone.gemfire.cache.lucene.internal;
 
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.execute.ResultCollector;
 import com.gemstone.gemfire.cache.lucene.LuceneQuery;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
@@ -55,7 +57,7 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
   }
 
   @Override
-  public LuceneQueryResults<K, V> search() {
+  public LuceneQueryResults<K, V> search() throws LuceneQueryException {
     TopEntriesCollectorManager manager = new TopEntriesCollectorManager(null, limit);
     LuceneFunctionContext<TopEntriesCollector> context = new LuceneFunctionContext<>(query, indexName, manager, limit);
     TopEntriesFunctionCollector collector = new TopEntriesFunctionCollector(context);
@@ -66,9 +68,16 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
         .execute(LuceneFunction.ID);
     
     //TODO provide a timeout to the user?
-    TopEntries entries = rc.getResult();
-    
-    return new LuceneQueryResultsImpl<K, V>(entries.getHits(), region, pageSize);
+    try {
+      TopEntries entries = rc.getResult();
+      return new LuceneQueryResultsImpl<K, V>(entries.getHits(), region, pageSize);
+    } catch(FunctionException e) {
+      if(e.getCause() instanceof LuceneQueryException) {
+        throw new LuceneQueryException(e);
+      } else {
+        throw e;
+      }
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
index 552278e..ca9f4f1 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
@@ -24,9 +24,12 @@ import static junitparams.JUnitParamsRunner.*;
 import static org.junit.Assert.*;
 
 import java.io.File;
+import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
 
+import junit.framework.AssertionFailedError;
+
 import com.jayway.awaitility.Awaitility;
 import junitparams.JUnitParamsRunner;
 import junitparams.Parameters;
@@ -92,7 +95,7 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
   }
 
   @Test
-  public void shouldRecoverPersistentIndexWhenDataStillInQueue() throws ParseException, InterruptedException {
+  public void shouldRecoverPersistentIndexWhenDataStillInQueue() throws Exception {
     createIndex(cache, "field1", "field2");
     Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
       .create(REGION_NAME);
@@ -113,7 +116,7 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
   }
 
   @Test
-  public void shouldRecoverPersistentIndexWhenDataIsWrittenToIndex() throws ParseException, InterruptedException {
+  public void shouldRecoverPersistentIndexWhenDataIsWrittenToIndex() throws Exception {
     createIndex(cache, "field1", "field2");
     Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
       .create(REGION_NAME);
@@ -132,7 +135,7 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
 
   @Test
   @Parameters(method = "getRegionShortcuts")
-  public void shouldHandleMultipleIndexes(RegionShortcut shortcut) throws ParseException {
+  public void shouldHandleMultipleIndexes(RegionShortcut shortcut) throws Exception {
     LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME+"_1", REGION_NAME, "field1");
     LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME+"_2", REGION_NAME, "field2");
     Region region = cache.createRegionFactory(shortcut).create(REGION_NAME);
@@ -159,10 +162,14 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
     });
   }
 
-  private void verifyQueryResultSize(String indexName, String regionName, String queryString, String defaultField, int size) throws ParseException {
+  private void verifyQueryResultSize(String indexName, String regionName, String queryString, String defaultField, int size) throws Exception {
     LuceneQuery query = luceneService.createLuceneQueryFactory().create(indexName, regionName, queryString, defaultField);
     Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
-      assertEquals(size, query.search().size());
+      try {
+        assertEquals(size, query.search().size());
+      } catch(LuceneQueryException e) {
+        throw new RuntimeException(e);
+      }
     });
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
index bfb8c88..7aa4cf5 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
@@ -17,10 +17,6 @@
 package com.gemstone.gemfire.cache.lucene;
 
 import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
-import static javax.swing.Action.DEFAULT;
-import static org.hamcrest.Matchers.isA;
-import static org.junit.Assert.assertEquals;
-
 import java.util.HashMap;
 import java.util.Map;
 
@@ -31,7 +27,6 @@ import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.apache.lucene.analysis.core.LowerCaseFilter;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.analysis.util.CharTokenizer;
-import org.apache.lucene.queryparser.classic.ParseException;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -39,9 +34,7 @@ import org.junit.rules.ExpectedException;
 
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.lucene.test.TestObject;
-import com.gemstone.gemfire.cache.query.QueryException;
 import com.gemstone.gemfire.pdx.JSONFormatter;
 import com.gemstone.gemfire.pdx.PdxInstance;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
@@ -57,7 +50,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   protected static final String REGION_NAME = "index";
 
   @Test()
-  public void shouldNotTokenizeWordsWithKeywordAnalyzer() throws ParseException {
+  public void shouldNotTokenizeWordsWithKeywordAnalyzer() throws Exception {
     Map<String, Analyzer> fields = new HashMap<String, Analyzer>();
     fields.put("field1", new StandardAnalyzer());
     fields.put("field2", new KeywordAnalyzer());
@@ -117,7 +110,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test()
-  public void shouldTokenizeUsingMyCharacterAnalyser() throws ParseException {
+  public void shouldTokenizeUsingMyCharacterAnalyser() throws Exception {
     Map<String, Analyzer> fields = new HashMap<String, Analyzer>();
     // not to specify field1's analyzer, it should use standard analyzer
     // Note: fields has to contain "field1", otherwise, field1 will not be tokenized
@@ -145,7 +138,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test()
-  public void shouldAllowNullInFieldValue() throws ParseException {
+  public void shouldAllowNullInFieldValue() throws Exception {
     Map<String, Analyzer> fields = new HashMap<String, Analyzer>();
     fields.put("field1", null);
     fields.put("field2", null);
@@ -163,7 +156,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test()
-  public void queryJsonObject() throws ParseException {
+  public void queryJsonObject() throws Exception {
     Map<String, Analyzer> fields = new HashMap<String, Analyzer>();
     fields.put("name", null);
     fields.put("lastName", null);
@@ -186,7 +179,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test()
-  public void shouldAllowQueryOnRegionWithStringValue() throws ParseException {
+  public void shouldAllowQueryOnRegionWithStringValue() throws Exception {
     luceneService.createIndex(INDEX_NAME, REGION_NAME, LuceneService.REGION_VALUE_FIELD);
     Region region = cache.createRegionFactory(RegionShortcut.PARTITION)
       .create(REGION_NAME);
@@ -199,7 +192,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test()
-  public void throwFunctionExceptionWhenGivenBadQuery() {
+  public void throwFunctionExceptionWhenGivenBadQuery() throws Exception {
     LuceneService luceneService = LuceneServiceProvider.get(cache);
     luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
     Region region = cache.createRegionFactory(RegionShortcut.PARTITION)
@@ -212,15 +205,8 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
       });
 
 
-    thrown.expect(FunctionException.class);
-    thrown.expectCause(isA(QueryException.class));
-    try {
-      query.search();
-    } catch(FunctionException e) {
-      assertEquals(LuceneQueryException.class, e.getCause().getClass());
-      throw e;
-    }
-
+    thrown.expect(LuceneQueryException.class);
+    query.search();
   }
   
   private PdxInstance insertAJson(Region region, String key) {
@@ -253,14 +239,14 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
     return pdx;
   }
 
-  private void verifyQuery(String query, String defaultField, String ... expectedKeys) throws ParseException {
+  private void verifyQuery(String query, String defaultField, String ... expectedKeys) throws Exception {
     final LuceneQuery<String, Object> queryWithStandardAnalyzer = luceneService.createLuceneQueryFactory().create(
       INDEX_NAME, REGION_NAME, query, defaultField);
 
     verifyQueryKeys(queryWithStandardAnalyzer, expectedKeys);
   }
   
-  private void verifyQuery(String query, String DEFAULT_FIELD, HashMap expectedResults) throws ParseException {
+  private void verifyQuery(String query, String DEFAULT_FIELD, HashMap expectedResults) throws Exception {
     final LuceneQuery<String, Object> queryWithStandardAnalyzer = luceneService.createLuceneQueryFactory().create(
       INDEX_NAME, REGION_NAME, query, DEFAULT_FIELD);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
index c2e9b0c..f748b79 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
@@ -61,7 +61,7 @@ public class LuceneQueryImplIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test
-  public void shouldInvokeLuceneFunctionWithCorrectArguments() {
+  public void shouldInvokeLuceneFunctionWithCorrectArguments() throws Exception {
     // Register a fake function to observe the function invocation
     FunctionService.unregisterFunction(LuceneFunction.ID);
     TestLuceneFunction function = new TestLuceneFunction();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
index 178fae2..26c4d62 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
@@ -31,6 +31,7 @@ import java.util.List;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.execute.ResultSender;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
 import com.gemstone.gemfire.cache.lucene.internal.InternalLuceneService;
@@ -240,7 +241,7 @@ public class LuceneFunctionJUnitTest {
     when(mockContext.getDataSet()).thenReturn(mockRegion);
     when(mockContext.getArguments()).thenReturn(searchArgs);
     when(mockContext.<TopEntriesCollector>getResultSender()).thenReturn(mockResultSender);
-    when(queryProvider.getQuery(eq(mockIndex))).thenThrow(QueryException.class);
+    when(queryProvider.getQuery(eq(mockIndex))).thenThrow(LuceneQueryException.class);
     LuceneFunction function = new LuceneFunction();
 
     function.execute(mockContext);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
index d7150d9..c1f540d 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
@@ -33,6 +33,7 @@ import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.lucene.LuceneIndex;
 import com.gemstone.gemfire.cache.lucene.LuceneQuery;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
 import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
@@ -90,7 +91,7 @@ public class LuceneTestUtilities {
   /**
    * Verify that a query returns the expected list of keys. Ordering is ignored.
    */
-  public static <K> void  verifyQueryKeys(LuceneQuery<K,Object> query,K ... expectedKeys) {
+  public static <K> void  verifyQueryKeys(LuceneQuery<K,Object> query,K ... expectedKeys) throws LuceneQueryException {
     Set<K> expectedKeySet = new HashSet<>(Arrays.asList(expectedKeys));
     Set<K> actualKeySet = new HashSet<>();
     final LuceneQueryResults<K, Object> results = query.search();
@@ -104,7 +105,7 @@ public class LuceneTestUtilities {
   /**
    * Verify that a query returns the expected map of key-value. Ordering is ignored.
    */
-  public static <K> void verifyQueryKeyAndValues(LuceneQuery<K,Object> query, HashMap expectedResults) {
+  public static <K> void verifyQueryKeyAndValues(LuceneQuery<K,Object> query, HashMap expectedResults) throws LuceneQueryException {
     HashMap actualResults = new HashMap<>();
     final LuceneQueryResults<K, Object> results = query.search();
     while(results.hasNextPage()) {


[06/16] incubator-geode git commit: GEODE-1209: Changes to ignoreEvictionAndExpiration AEQ attribute based on the new proposal (changing ignoreEvictionAndExpiration to forwardExpirationDestroy).

Posted by kl...@apache.org.
GEODE-1209: Changes to ignoreEvictionAndExpiration AEQ attribute based on the new proposal
(changing ignoreEvictionAndExpiration to forwardExpirationDestroy).

GEODE-1209 was proposed to add new attribute ignoreEvictionAndExpiration to AsyncEventQueue.
As mentioned in the ticket due to product issue a new proposal was made to change the
functionality, to only forward expiration-destroy operation.

The changes made here replaces ignoreEvictionAndExpiration attribute to forwardExiprationDestroy.


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

Branch: refs/heads/feature/GEODE-1565
Commit: 52a13e8295bdced8bfc72ab7a710f2a4a8df0ca7
Parents: 2fe3a4b
Author: Anil <ag...@pivotal.io>
Authored: Thu Jun 16 18:07:13 2016 -0700
Committer: Anil <ag...@pivotal.io>
Committed: Thu Jun 16 18:16:10 2016 -0700

----------------------------------------------------------------------
 .../SharedConfigurationEndToEndDUnitTest.java   |   4 +-
 .../cache/asyncqueue/AsyncEventQueue.java       |  14 +-
 .../asyncqueue/AsyncEventQueueFactory.java      |  16 +-
 .../internal/AsyncEventQueueFactoryImpl.java    |   6 +-
 .../internal/AsyncEventQueueImpl.java           |   8 +-
 .../gemfire/cache/wan/GatewaySender.java        |   8 +-
 .../cache/wan/AbstractGatewaySender.java        |  49 ++-
 .../cache/wan/GatewaySenderAttributes.java      |  12 +-
 .../cache/xmlcache/AsyncEventQueueCreation.java |  20 +-
 .../internal/cache/xmlcache/CacheXml.java       |   4 +-
 .../cache/xmlcache/CacheXmlGenerator.java       |   6 +-
 .../internal/cache/xmlcache/CacheXmlParser.java |  16 +-
 .../internal/cli/commands/QueueCommands.java    |  20 +-
 .../functions/AsyncEventQueueFunctionArgs.java  |  14 +-
 .../CreateAsyncEventQueueFunction.java          |  10 +-
 .../internal/cli/i18n/CliStrings.java           |   4 +-
 .../controllers/QueueCommandsController.java    |   4 +-
 .../geode.apache.org/schema/cache/cache-1.0.xsd |   2 +-
 ...ventQueueEvictionAndExpirationJUnitTest.java | 362 +++++++++++++------
 .../cache30/CacheXmlGeode10DUnitTest.java       |  23 +-
 .../cli/commands/QueueCommandsDUnitTest.java    |   4 +-
 .../codeAnalysis/sanctionedSerializables.txt    |   4 +-
 .../cli/commands/golden-help-offline.properties |  10 +-
 23 files changed, 375 insertions(+), 245 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
index 899f187..3408717 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/configuration/SharedConfigurationEndToEndDUnitTest.java
@@ -227,12 +227,12 @@ public class SharedConfigurationEndToEndDUnitTest extends CliCommandTestBase {
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISPATCHERTHREADS, "4");
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ENABLEBATCHCONFLATION, "true");
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS, "true");
-      csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION, "true");
+      csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY, "false");
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY, "1000");
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ORDERPOLICY, OrderPolicy.KEY.toString());
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__PERSISTENT, "true");
       csb.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__PARALLEL, "true");
-      
+
       executeAndVerifyCommand(csb.getCommandString());
 
     } finally {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueue.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueue.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueue.java
index edf887b..85ca392 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueue.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueue.java
@@ -147,13 +147,13 @@ public interface AsyncEventQueue {
    *         <code>AsyncEventQueue</code>
    */
   public GatewayEventSubstitutionFilter getGatewayEventSubstitutionFilter();
- 
+
   /**
-   * Represents if eviction and expiration events/operations are ignored (not passed)
-   * with <code>AsyncEventListener</code>.
-   * 
-   * @return boolen True if eviction and expiration operations are ignored.
+   * Represents if expiration destroy operations are forwarded (passed)
+   * to <code>AsyncEventListener</code>.
+   *
+   * @return boolean True if expiration destroy operations are forwarded.
    */
-  public boolean isIgnoreEvictionAndExpiration();
-  
+  public boolean isForwardExpirationDestroy();
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java
index 6294dfe..455fc14 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java
@@ -171,19 +171,15 @@ public interface AsyncEventQueueFactory {
       GatewayEventSubstitutionFilter filter);
 
   /**
-   * Ignores the eviction and expiration events.
-   * By default its set to ignore eviction and expiration events (true), by
-   * setting it to false, the AEQ will receive destroy events due to eviction
-   * and expiration action.
-   * Note, setting this to false doesn't propagate invalidate events due to
-   * expiration action.
+   * Forwards destroy operations from expiration action to AsynEventQueue.
+   * By default the expiration destroy events are not added to AEQ, by
+   * setting this attribute to true adds destroy event to AEQ.
    *
-   * @param ignore 
-   *        boolean to indicate whether to ignore eviction and expiration events. 
+   * @param forward
+   *        boolean to indicate whether to forward expiration destroy events.
    */
-  public AsyncEventQueueFactory setIgnoreEvictionAndExpiration(boolean ignore);
+  public AsyncEventQueueFactory setForwardExpirationDestroy(boolean forward);
 
-  
   /**
    * Creates the <code>AsyncEventQueue</code>. It accepts Id of AsyncEventQueue
    * and instance of AsyncEventListener. Multiple queues can be created using

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
index 1ec3ba0..f658621 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueFactoryImpl.java
@@ -277,7 +277,7 @@ public class AsyncEventQueueFactoryImpl implements AsyncEventQueueFactory {
     this.attrs.eventFilters = asyncQueueCreation.getGatewayEventFilters();
     this.attrs.eventSubstitutionFilter = asyncQueueCreation.getGatewayEventSubstitutionFilter();
     this.attrs.isForInternalUse = true;
-    this.attrs.ignoreEvictionAndExpiration = asyncQueueCreation.isIgnoreEvictionAndExpiration();
+    this.attrs.forwardExpirationDestroy = asyncQueueCreation.isForwardExpirationDestroy();
   }
 
   public AsyncEventQueueFactory setParallel(boolean isParallel) {
@@ -294,8 +294,8 @@ public class AsyncEventQueueFactoryImpl implements AsyncEventQueueFactory {
   }
 
   @Override
-  public AsyncEventQueueFactory setIgnoreEvictionAndExpiration(boolean ignore) {
-    this.attrs.ignoreEvictionAndExpiration = ignore;
+  public AsyncEventQueueFactory setForwardExpirationDestroy(boolean forward) {
+    this.attrs.forwardExpirationDestroy = forward;
     return this;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueImpl.java
index 5a0b370..994bbfc 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/internal/AsyncEventQueueImpl.java
@@ -200,9 +200,9 @@ public class AsyncEventQueueImpl implements AsyncEventQueue {
    public boolean isBucketSorted() {
     // TODO Auto-generated method stub
     return false;
-  }     
-   
-   public boolean isIgnoreEvictionAndExpiration() {
-     return ((AbstractGatewaySender)this.sender).isIgnoreEvictionAndExpiration();
+  }
+
+   public boolean isForwardExpirationDestroy() {
+     return ((AbstractGatewaySender)this.sender).isForwardExpirationDestroy();
    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java
index d559a1a..04dc248 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/wan/GatewaySender.java
@@ -93,13 +93,13 @@ public interface GatewaySender {
       .getInteger(
           DistributionConfig.GEMFIRE_PREFIX + "cache.gatewaySender.defaultParallelismForReplicatedRegion",
           113).intValue();  
-  
+
   public static final int DEFAULT_DISTRIBUTED_SYSTEM_ID = -1;
 
   public static final int DEFAULT_DISPATCHER_THREADS = 5;
-  
-  public static final boolean DEFAULT_IGNORE_EVICTION_EXPIRATION = true;
-  
+
+  public static final boolean DEFAULT_FORWARD_EXPIRATION_DESTROY = false;
+
   public static final OrderPolicy DEFAULT_ORDER_POLICY = OrderPolicy.KEY;
   /**
    * The default maximum amount of memory (MB) to allow in the queue before

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/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 7e2a0af..06232a3 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
@@ -137,9 +137,9 @@ public abstract class AbstractGatewaySender implements GatewaySender,
   protected List<GatewayTransportFilter> transFilters;
 
   protected List<AsyncEventListener> listeners;
-  
-  protected boolean ignoreEvictionAndExpiration;
-  
+
+  protected boolean forwardExpirationDestroy;
+
   protected GatewayEventSubstitutionFilter substitutionFilter;
   
   protected LocatorDiscoveryCallback locatorDiscoveryCallback;
@@ -272,9 +272,9 @@ public abstract class AbstractGatewaySender implements GatewaySender,
       initializeEventIdIndex();
     }
     this.isBucketSorted = attrs.isBucketSorted();
-    this.ignoreEvictionAndExpiration = attrs.isIgnoreEvictionAndExpiration();
+    this.forwardExpirationDestroy = attrs.isForwardExpirationDestroy();
   }
-  
+
   public GatewaySenderAdvisor getSenderAdvisor() {
     return senderAdvisor;
   }
@@ -346,11 +346,11 @@ public abstract class AbstractGatewaySender implements GatewaySender,
   public boolean hasListeners() {
     return !this.listeners.isEmpty();
   }
-  
-  public boolean isIgnoreEvictionAndExpiration() {
-    return this.ignoreEvictionAndExpiration;
+
+  public boolean isForwardExpirationDestroy() {
+    return this.forwardExpirationDestroy;
   }
-  
+
   public boolean isManualStart() {
     return this.manualStart;
   }
@@ -809,26 +809,23 @@ public abstract class AbstractGatewaySender implements GatewaySender,
     {
       return false;
     }
-    
-    // Eviction and expirations are not passed to WAN.
-    // Eviction and Expiration are passed to AEQ based on its configuration.
+    // Check for eviction and expiration events.
     if (event.getOperation().isLocal() || event.getOperation().isExpiration()) {
-      // Check if its AEQ and AEQ is configured to forward eviction/expiration events.
-      if (this.isAsyncEventQueue() && !this.isIgnoreEvictionAndExpiration()) {
+      // Check if its AEQ and is configured to forward expiration destroy events.
+      if (event.getOperation().isExpiration() && this.isAsyncEventQueue() && this.isForwardExpirationDestroy()) {
         return true;
       }
       return false;
     }
-    
     return true;
   }
-  
-  
+
+
   public void distribute(EnumListenerEvent operation, EntryEventImpl event,
       List<Integer> allRemoteDSIds) {
-    
+
     final boolean isDebugEnabled = logger.isDebugEnabled();
-    
+
     // If this gateway is not running, return
     if (!isRunning()) {
       if (isDebugEnabled) {
@@ -836,20 +833,20 @@ public abstract class AbstractGatewaySender implements GatewaySender,
       }
       return;
     }
-    
+
     final GatewaySenderStats stats = getStatistics();
     stats.incEventsReceived();
-   
+
     if (!checkForDistribution(event, stats)) {
-      getStatistics().incEventsNotQueued();
+      stats.incEventsNotQueued();
       return;
     }
-    
+
     // this filter is defined by Asif which exist in old wan too. new wan has
     // other GatewaEventFilter. Do we need to get rid of this filter. Cheetah is
-    // not cinsidering this filter
+    // not considering this filter
     if (!this.filter.enqueueEvent(event)) {
-      getStatistics().incEventsFiltered();
+      stats.incEventsFiltered();
       return;
     }
     // released by this method or transfers ownership to TmpQueueEvent
@@ -861,7 +858,7 @@ public abstract class AbstractGatewaySender implements GatewaySender,
 
     setModifiedEventId(clonedEvent);
     Object callbackArg = clonedEvent.getRawCallbackArgument();
-    
+
     if (isDebugEnabled) {
       // We can't deserialize here for logging purposes so don't
       // call getNewValue.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderAttributes.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderAttributes.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderAttributes.java
index 163943f..d023dfc 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderAttributes.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/GatewaySenderAttributes.java
@@ -82,9 +82,9 @@ public class GatewaySenderAttributes {
   public boolean isBucketSorted = GatewaySenderAttributes.DEFAULT_IS_BUCKETSORTED;
   
   public boolean isMetaQueue = GatewaySenderAttributes.DEFAULT_IS_META_QUEUE;
-  
-  public boolean ignoreEvictionAndExpiration = GatewaySender.DEFAULT_IGNORE_EVICTION_EXPIRATION;
-  
+
+  public boolean forwardExpirationDestroy = GatewaySender.DEFAULT_FORWARD_EXPIRATION_DESTROY;
+
   public int getSocketBufferSize() {
     return this.socketBufferSize;
   }
@@ -194,9 +194,9 @@ public class GatewaySenderAttributes {
   public boolean isMetaQueue() {
     return this.isMetaQueue;
   }
-  
-  public boolean isIgnoreEvictionAndExpiration() {
-    return this.ignoreEvictionAndExpiration;
+
+  public boolean isForwardExpirationDestroy() {
+    return this.forwardExpirationDestroy;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/AsyncEventQueueCreation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/AsyncEventQueueCreation.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/AsyncEventQueueCreation.java
index e55ec3f..e5803c1 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/AsyncEventQueueCreation.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/AsyncEventQueueCreation.java
@@ -43,11 +43,11 @@ public class AsyncEventQueueCreation implements AsyncEventQueue {
   private boolean isBucketSorted = false;
   private int dispatcherThreads = 1;
   private OrderPolicy orderPolicy = OrderPolicy.KEY;
-  private boolean ignoreEvictionAndExpiration = true;
-  
+  private boolean forwardExpirationDestroy = false;
+
   public AsyncEventQueueCreation() {
   }
-  
+
   public AsyncEventQueueCreation(String id, GatewaySenderAttributes senderAttrs, AsyncEventListener eventListener) {
     this.id = id;
     this.batchSize = senderAttrs.batchSize;
@@ -64,9 +64,9 @@ public class AsyncEventQueueCreation implements AsyncEventQueue {
     this.isBucketSorted = senderAttrs.isBucketSorted;
     this.gatewayEventFilters = senderAttrs.eventFilters;
     this.gatewayEventSubstitutionFilter = senderAttrs.eventSubstitutionFilter;
-    this.ignoreEvictionAndExpiration = senderAttrs.ignoreEvictionAndExpiration;
+    this.forwardExpirationDestroy = senderAttrs.forwardExpirationDestroy;
   }
-  
+
   @Override
   public AsyncEventListener getAsyncEventListener() {
     return this.asyncEventListener;
@@ -215,12 +215,12 @@ public class AsyncEventQueueCreation implements AsyncEventQueue {
     this.isBucketSorted = isBucketSorted;
   }
 
-  public void setIgnoreEvictionAndExpiration(boolean ignore) {
-    this.ignoreEvictionAndExpiration = ignore;
+  public void setForwardExpirationDestroy(boolean forward) {
+    this.forwardExpirationDestroy = forward;
   }
-  
+
   @Override
-  public boolean isIgnoreEvictionAndExpiration() {
-    return this.ignoreEvictionAndExpiration;
+  public boolean isForwardExpirationDestroy() {
+    return this.forwardExpirationDestroy;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXml.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXml.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXml.java
index 71171fd..1671d4e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXml.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXml.java
@@ -757,8 +757,8 @@ public abstract class CacheXml implements EntityResolver2, ErrorHandler {
   protected static final String ASYNC_EVENT_LISTENER = "async-event-listener";
   public static final String ASYNC_EVENT_QUEUE = "async-event-queue";
   protected static final String ASYNC_EVENT_QUEUE_IDS = "async-event-queue-ids";
-  protected static final String IGNORE_EVICTION_AND_EXPIRATION = "ignore-eviction-expiration";
-  
+  protected static final String FORWARD_EXPIRATION_DESTROY = "forward-expiration-destroy";
+
   /** The name of the <code>compressor</code> attribute */
   protected static final String COMPRESSOR = "compressor";
   /** The name of the <code>off-heap</code> attribute

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java
index 17076db..8f604ba 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlGenerator.java
@@ -1523,9 +1523,9 @@ public class CacheXmlGenerator extends CacheXml implements XMLReader {
       }
       // eviction and expiration events
       if (this.version.compareTo(CacheXmlVersion.GEODE_1_0) >= 0) {
-        if (generateDefaults() || asyncEventQueue.isIgnoreEvictionAndExpiration() != (GatewaySender.DEFAULT_IGNORE_EVICTION_EXPIRATION))
-          atts.addAttribute("", "", IGNORE_EVICTION_AND_EXPIRATION, "", String.valueOf(asyncEventQueue
-              .isIgnoreEvictionAndExpiration()));
+        if (generateDefaults() || asyncEventQueue.isForwardExpirationDestroy() != (GatewaySender.DEFAULT_FORWARD_EXPIRATION_DESTROY))
+          atts.addAttribute("", "", FORWARD_EXPIRATION_DESTROY, "", String.valueOf(asyncEventQueue
+              .isForwardExpirationDestroy()));
       }
       // disk-synchronous
       if (generateDefaults() || asyncEventQueue.isDiskSynchronous() != GatewaySender.DEFAULT_DISK_SYNCHRONOUS)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/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 76ab0f9..0564cb5 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
@@ -2312,16 +2312,16 @@ public class CacheXmlParser extends CacheXml implements ContentHandler {
                 .toLocalizedString(new Object[] { id, orderPolicy }));
       }
     }
-    
-    // forward eviction and expiration events.
-    String ignoreEvictionExpiration = atts.getValue(IGNORE_EVICTION_AND_EXPIRATION);
-    if (ignoreEvictionExpiration != null) {
-      asyncEventQueueCreation.setIgnoreEvictionAndExpiration(Boolean.parseBoolean(ignoreEvictionExpiration));
+
+    // forward expiration destroy events.
+    String forward = atts.getValue(FORWARD_EXPIRATION_DESTROY);
+    if (forward != null) {
+      asyncEventQueueCreation.setForwardExpirationDestroy(Boolean.parseBoolean(forward));
     }
-    
+
     stack.push(asyncEventQueueCreation);
   }
-  
+
   private void endAsyncEventListener() {
     Declarable d = createDeclarable();
     if (!(d instanceof AsyncEventListener)) {
@@ -2352,7 +2352,7 @@ public class CacheXmlParser extends CacheXml implements ContentHandler {
     factory.setMaximumQueueMemory(asyncEventChannelCreation.getMaximumQueueMemory());
     factory.setDispatcherThreads(asyncEventChannelCreation.getDispatcherThreads());
     factory.setOrderPolicy(asyncEventChannelCreation.getOrderPolicy());
-    factory.setIgnoreEvictionAndExpiration(asyncEventChannelCreation.isIgnoreEvictionAndExpiration());
+    factory.setForwardExpirationDestroy(asyncEventChannelCreation.isForwardExpirationDestroy());
     List<GatewayEventFilter> gatewayEventFilters = asyncEventChannelCreation.getGatewayEventFilters();
     for (GatewayEventFilter gatewayEventFilter : gatewayEventFilters) {
       factory.addGatewayEventFilter(gatewayEventFilter);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java
index d84959f..c321a0d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java
@@ -105,15 +105,15 @@ public class QueueCommands implements CommandMarker {
       @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS,
                  unspecifiedDefaultValue = "true",
                  specifiedDefaultValue = "true",
-                 help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS__HELP) 
+                 help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS__HELP)
       Boolean diskSynchronous,
-      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION,
-                 unspecifiedDefaultValue = "true",
-                 specifiedDefaultValue = "true",
-                 help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION__HELP) 
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY,
+                 unspecifiedDefaultValue = "false",
+                 specifiedDefaultValue = "false",
+                 help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY__HELP)
       Boolean ignoreEvictionAndExpiration,
       @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY,
-                 unspecifiedDefaultValue = "100", 
+                 unspecifiedDefaultValue = "100",
                  help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY__HELP)
       int maxQueueMemory,
       @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISPATCHERTHREADS,
@@ -166,13 +166,13 @@ public class QueueCommands implements CommandMarker {
         return crex.getResult();
       }
 
-      AsyncEventQueueFunctionArgs aeqArgs = new AsyncEventQueueFunctionArgs(id, parallel, 
+      AsyncEventQueueFunctionArgs aeqArgs = new AsyncEventQueueFunctionArgs(id, parallel,
           enableBatchConflation, batchSize,batchTimeInterval,
-          persistent, diskStore, diskSynchronous, maxQueueMemory, dispatcherThreads, orderPolicy, 
-          gatewayEventFilters, gatewaySubstitutionListener, listener, listenerProperties, 
+          persistent, diskStore, diskSynchronous, maxQueueMemory, dispatcherThreads, orderPolicy,
+          gatewayEventFilters, gatewaySubstitutionListener, listener, listenerProperties,
           ignoreEvictionAndExpiration);
 
-      ResultCollector<?, ?> rc = CliUtil.executeFunction(new CreateAsyncEventQueueFunction(), 
+      ResultCollector<?, ?> rc = CliUtil.executeFunction(new CreateAsyncEventQueueFunction(),
           aeqArgs, targetMembers);
 
       List<CliFunctionResult> results = CliFunctionResult.cleanResults((List<?>) rc.getResult());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs.java
index 2066628..a4a040b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs.java
@@ -23,7 +23,7 @@ import java.util.Properties;
  * This class stores the arguments provided for create async event queue command.
  */
 public class AsyncEventQueueFunctionArgs implements Serializable {
-  
+
   private static final long serialVersionUID = -6524494645663740872L;
 
   private String asyncEventQueueId;
@@ -35,13 +35,13 @@ public class AsyncEventQueueFunctionArgs implements Serializable {
   private String diskStoreName;
   private boolean diskSynchronous;
   private int maxQueueMemory;
-  private int dispatcherThreads; 
+  private int dispatcherThreads;
   private String orderPolicy;
   private String[] gatewayEventFilters;
   private String gatewaySubstitutionFilter;
   private String listenerClassName;
   private Properties listenerProperties;
-  private boolean ignoreEvictionAndExpiration;
+  private boolean forwardExpirationDestroy;
 
   public AsyncEventQueueFunctionArgs(String asyncEventQueueId,
       boolean isParallel, boolean enableBatchConflation, int batchSize,
@@ -49,7 +49,7 @@ public class AsyncEventQueueFunctionArgs implements Serializable {
       boolean diskSynchronous, int maxQueueMemory, int dispatcherThreads,
       String orderPolicy, String[] gatewayEventFilters,
       String gatewaySubstitutionFilter, String listenerClassName,
-      Properties listenerProperties, boolean ignoreEvictionAndExpiration) {
+      Properties listenerProperties, boolean forwardExpirationDestroy) {
     this.asyncEventQueueId = asyncEventQueueId;
     this.isParallel = isParallel;
     this.enableBatchConflation = enableBatchConflation;
@@ -65,7 +65,7 @@ public class AsyncEventQueueFunctionArgs implements Serializable {
     this.gatewaySubstitutionFilter = gatewaySubstitutionFilter;
     this.listenerClassName = listenerClassName;
     this.listenerProperties = listenerProperties;
-    this.ignoreEvictionAndExpiration = ignoreEvictionAndExpiration;
+    this.forwardExpirationDestroy = forwardExpirationDestroy;
   }
 
   public String getAsyncEventQueueId() {
@@ -128,7 +128,7 @@ public class AsyncEventQueueFunctionArgs implements Serializable {
     return listenerProperties;
   }
 
-  public boolean isIgnoreEvictionAndExpiration() {
-    return ignoreEvictionAndExpiration;
+  public boolean isForwardExpirationDestroy() {
+    return forwardExpirationDestroy;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java
index 32e8f83..c6cd8fa 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java
@@ -63,7 +63,7 @@ public class CreateAsyncEventQueueFunction extends FunctionAdapter implements In
 
     try {
       AsyncEventQueueFunctionArgs aeqArgs =  (AsyncEventQueueFunctionArgs)context.getArguments();
-      
+
       GemFireCacheImpl cache = (GemFireCacheImpl) CacheFactory.getAnyInstance();
 
       DistributedMember member = cache.getDistributedSystem().getDistributedMember();
@@ -82,7 +82,7 @@ public class CreateAsyncEventQueueFunction extends FunctionAdapter implements In
           .setPersistent(aeqArgs.isPersistent())
           .setDiskStoreName(aeqArgs.getDiskStoreName())
           .setDiskSynchronous(aeqArgs.isDiskSynchronous())
-          .setIgnoreEvictionAndExpiration(aeqArgs.isIgnoreEvictionAndExpiration())
+          .setForwardExpirationDestroy(aeqArgs.isForwardExpirationDestroy())
           .setMaximumQueueMemory(aeqArgs.getMaxQueueMemory())
           .setDispatcherThreads(aeqArgs.getDispatcherThreads())
           .setOrderPolicy(OrderPolicy.valueOf(aeqArgs.getOrderPolicy()));
@@ -94,13 +94,13 @@ public class CreateAsyncEventQueueFunction extends FunctionAdapter implements In
           asyncEventQueueFactory.addGatewayEventFilter((GatewayEventFilter) newInstance(gatewayEventFilterKlass, CliStrings.CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER));
         }
       }
-      
+
       String gatewaySubstitutionFilter = aeqArgs.getGatewaySubstitutionFilter();
       if (gatewaySubstitutionFilter != null) {
         Class<?> gatewayEventSubstitutionFilterKlass = forName(gatewaySubstitutionFilter, CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER);
         asyncEventQueueFactory.setGatewayEventSubstitutionListener((GatewayEventSubstitutionFilter<?,?>) newInstance(gatewayEventSubstitutionFilterKlass, CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER));
       }
-    
+
       String listenerClassName = aeqArgs.getListenerClassName();
       Object listenerInstance;
       Class<?> listenerClass = InternalDataSerializer.getCachedClass(listenerClassName);
@@ -111,7 +111,7 @@ public class CreateAsyncEventQueueFunction extends FunctionAdapter implements In
         if (!(listenerInstance instanceof Declarable)) {
           throw new IllegalArgumentException("Listener properties were provided, but the listener specified does not implement Declarable.");
         }
-        
+
         ((Declarable) listenerInstance).init(listenerProperties);
 
         Map<Declarable, Properties> declarablesMap = new HashMap<Declarable, Properties>();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
index 787edcc..cff6535 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
@@ -426,8 +426,8 @@ public class CliStrings {
   public static final String CREATE_ASYNC_EVENT_QUEUE__DISK_STORE__HELP = "Disk store to be used by this queue.";
   public static final String CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS = "disk-synchronous";
   public static final String CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS__HELP = "Whether disk writes are synchronous.";
-  public static final String CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION = "ignore-eviction-expiration";
-  public static final String CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION__HELP = "Whether to ignore eviction and expiration events.";
+  public static final String CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY = "forward-expiration-destroy";
+  public static final String CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY__HELP = "Whether to forward expiration destroy events.";
   public static final String CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY = "max-queue-memory";
   public static final String CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY__HELP = "Maximum amount of memory, in megabytes, that the queue can consume before overflowing to disk.";
   public static final String CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER = "gateway-event-filter";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java
index 9367612..387ea4e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java
@@ -58,7 +58,7 @@ public class QueueCommandsController extends AbstractCommandsController {
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__PERSISTENT, defaultValue = "false") final Boolean persistent,
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISK_STORE, required = false) final String diskStore,
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS, defaultValue = "true") final Boolean diskSynchronous,
-                                      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION, defaultValue = "true") final Boolean isIgnoreEvictionAndExpiration,
+                                      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY, defaultValue = "false") final Boolean forwardExpirationDestroy,
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__MAXIMUM_QUEUE_MEMORY, defaultValue = "100") final Integer maxQueueMemory,
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISPATCHERTHREADS, defaultValue = "1") final Integer dispatcherThreads,
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__ORDERPOLICY, defaultValue = "KEY") final String orderPolicy,
@@ -83,7 +83,7 @@ public class QueueCommandsController extends AbstractCommandsController {
 
     command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__PARALLEL, String.valueOf(Boolean.TRUE.equals(parallel)));
     command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ENABLEBATCHCONFLATION, String.valueOf(Boolean.TRUE.equals(enableBatchConflation)));
-    command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION, String.valueOf(isIgnoreEvictionAndExpiration));
+    command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY, String.valueOf(forwardExpirationDestroy));
 
     if (hasValue(batchSize)) {
       command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__BATCH_SIZE, String.valueOf(batchSize));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/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 688ff1f..452baa3 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
@@ -254,7 +254,7 @@ declarative caching XML file elements unless indicated otherwise.
             <xsd:attribute name="disk-synchronous" type="xsd:boolean" use="optional" />
             <xsd:attribute name="dispatcher-threads" type="xsd:string" use="optional" />
             <xsd:attribute name="order-policy" type="xsd:string" use="optional" />
-            <xsd:attribute default="true" name="ignore-eviction-expiration" type="xsd:boolean" use="optional" />
+            <xsd:attribute default="false" name="forward-expiration-destroy" type="xsd:boolean" use="optional" />
           </xsd:complexType>
         </xsd:element>
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java
index 9add46b..c37810f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java
@@ -17,9 +17,14 @@
 package com.gemstone.gemfire.cache.asyncqueue;
 
 import com.gemstone.gemfire.cache.*;
+import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 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.wan.AbstractGatewaySender;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import com.jayway.awaitility.Awaitility;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -32,6 +37,7 @@ import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import static com.gemstone.gemfire.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Mockito.mock;
@@ -41,12 +47,16 @@ public class AsyncEventQueueEvictionAndExpirationJUnitTest {
   
   private AsyncEventQueue aeq;
   private Cache cache;
+  private Region region;
+  String aeqId;
+  List<AsyncEvent> events = new ArrayList<AsyncEvent>();
   
   @Rule 
   public TestName name = new TestName();
   
   @Before
-  public void getCache() {
+  public void setup() {
+    events.clear();
     try {
        cache = CacheFactory.getAnyInstance();
     } catch (Exception e) {
@@ -55,10 +65,11 @@ public class AsyncEventQueueEvictionAndExpirationJUnitTest {
     if (null == cache) {
       cache = (GemFireCacheImpl) new CacheFactory().set(MCAST_PORT, "0").create();
     }
+    aeqId = name.getMethodName();
   }
 
   @After
-  public void destroyCache() {
+  public void tearDown() {
     if (cache != null && !cache.isClosed()) {
       cache.close();
       cache = null;
@@ -67,19 +78,19 @@ public class AsyncEventQueueEvictionAndExpirationJUnitTest {
 
   
   @Test
-  public void isIgnoreEvictionAndExpirationAttributeTrueByDefault() {
+  public void isForwardExpirationDestroyAttributeFalseByDefault() {
     AsyncEventListener al = mock(AsyncEventListener.class);
     aeq = cache.createAsyncEventQueueFactory().create("aeq", al);
     // Test for default value of isIgnoreEvictionAndExpiration setting.
-    assertTrue(aeq.isIgnoreEvictionAndExpiration());
+    assertFalse(aeq.isForwardExpirationDestroy());
   }
   
   @Test
-  public void canSetFalseForIgnoreEvictionAndExpiration() {
+  public void canSetTrueForForwardExpirationDestroy() {
     AsyncEventListener al = mock(AsyncEventListener.class);
-    aeq = cache.createAsyncEventQueueFactory().setIgnoreEvictionAndExpiration(false).create("aeq", al);
+    aeq = cache.createAsyncEventQueueFactory().setForwardExpirationDestroy(true).create("aeq", al);
     // Test for default value of isIgnoreEvictionAndExpiration setting.
-    assertFalse(aeq.isIgnoreEvictionAndExpiration());
+    assertTrue(aeq.isForwardExpirationDestroy());
   }
   
   
@@ -87,182 +98,298 @@ public class AsyncEventQueueEvictionAndExpirationJUnitTest {
   public void evictionDestroyOpEventsNotPropogatedByDefault() {
     // For Replicated Region with eviction-destroy op.
     // Number of expected events 2. Two for create and none for eviction destroy.
-    createPopulateAndVerifyEvents(false /*isPR */, true /* ignoreEvictionExpiration */, 
-        2 /* expectedEvents */ , true /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, false /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(false /*isPR */, false /* forwardExpirationDestroy */, 
+        true /* eviction */, false /* evictionOverflow */, 
+        false /* expirationDestroy */, false /* expirationInvalidate */);
+
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, region.size()));
+
+    // Validate events that are not queued.
+    // This guarantees that eviction/expiration is performed and events are
+    // sent all the way to Gateway.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, getEventsNotQueuedSize(aeqId)));
+
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
   }
 
   @Test
   public void evictionDestroyOpEventsNotPropogatedByDefaultForPR() {
     // For PR with eviction-destroy op.
     // Number of expected events 2. Two for create and none for eviction destroy.
-    createPopulateAndVerifyEvents(true /*isPR */, true /* ignoreEvictionExpiration */, 
-        2 /* expectedEvents */ , true /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, false /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(true /*isPR */, false /* forwardExpirationDestroy */, 
+        true /* eviction */, false /* evictionOverflow */,
+        false /* expirationDestroy */, false /* expirationInvalidate */);
+
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, region.size()));
+
+   // Validate events that are not queued.
+   // This guarantees that eviction/expiration is performed and events are
+   // sent all the way to Gateway.
+   // In case of eviction one event is evicted that should not be queued.
+   Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, getEventsNotQueuedSize(aeqId)));
+
+   // The AQListner should get expected events.
+   Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+    
   }
 
   @Test
   public void expirationDestroyOpEventsNotPropogatedByDefault() {
     // For Replicated Region with expiration-destroy op.
     // Number of expected events 2. Two for create and none for eviction destroy.
-    createPopulateAndVerifyEvents(false /*isPR */, true /* ignoreEvictionExpiration */, 
-        2 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        true /* expirationDestroy */, false /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(false /*isPR */, false /* forwardExpirationDestroy */, 
+        false /* eviction */, false /* evictionOverflow */, 
+        true /* expirationDestroy */, false /* expirationInvalidate */);
+
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(0, region.size()));
+
+    // Validate events that are not queued.
+    // This guarantees that eviction/expiration is performed and events are
+    // sent all the way to Gateway.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, getEventsNotQueuedSize(aeqId)));
+
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
   }
 
   @Test
   public void expirationDestroyOpEventsNotPropogatedByDefaultForPR() {
     // For PR with expiration-destroy op.
-    // Number of expected events 2. Two for create and none for eviction destroy.
-    createPopulateAndVerifyEvents(true /*isPR */, true /* ignoreEvictionExpiration */, 
-        2 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        true /* expirationDestroy */, false /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    // Number of expected events 2. Two for create and none for eviction destroy.    
+    createRegionAeqAndPopulate(true /*isPR */, false /* forwardExpirationDestroy */, 
+        false /* eviction */, false /* evictionOverflow */, 
+        true /* expirationDestroy */, false /* expirationInvalidate */);
+
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(0, region.size()));
+
+    // Validate events that are not queued.
+    // This guarantees that eviction/expiration is performed and events are
+    // sent all the way to Gateway.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, getEventsNotQueuedSize(aeqId)));
+
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
   }
 
   @Test
   public void expirationInvalidOpEventsNotPropogatedByDefault() {
     // For Replicated Region with expiration-invalid op.
-    // Number of expected events 2. Two for create and none for eviction destroy.
-    createPopulateAndVerifyEvents(false /*isPR */, true /* ignoreEvictionExpiration */, 
-        2 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, true /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    // Number of expected events 2. Two for create and none for eviction destroy.    
+    createRegionAeqAndPopulate(false /*isPR */, false /* forwardExpirationDestroy */, 
+        false /* eviction */, false /* evictionOverflow */, 
+        false /* expirationDestroy */, true /* expirationInvalidate */);
+
+    LocalRegion lr = (LocalRegion)region;
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, lr.getCachePerfStats().getInvalidates()));
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
+    
   }
 
   @Test
   public void expirationInvalidOpEventsNotPropogatedByDefaultForPR() {
     // For Replicated Region with expiration-invalid op.
     // Number of expected events 2. Two for create and none for eviction destroy.
-    createPopulateAndVerifyEvents(true /*isPR */, true /* ignoreEvictionExpiration */, 
-        2 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, true /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(true /*isPR */, false /* forwardExpirationDestroy */, 
+        false /* eviction */, false /* evictionOverflow */, 
+        false /* expirationDestroy */, true /* expirationInvalidate */);
+
+    LocalRegion lr = (LocalRegion)region;
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, lr.getCachePerfStats().getInvalidates()));
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
   }
   
   @Test
-  public void evictionPropogatedUsingIgnoreEvictionAndExpirationAttribute() {
+  public void evictionNotPropogatedUsingForwardExpirationDestroyAttribute() {
     // For Replicated Region with eviction-destroy op.
-    // Number of expected events 3. Two for create and One for eviction destroy.
-    createPopulateAndVerifyEvents(false /*isPR */, false /* ignoreEvictionExpiration */, 
-        3 /* expectedEvents */ , true /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, false /* expirationInvalidate */, 
-        true /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    // Number of expected events 2. Two for create and none for eviction destroy.
+    createRegionAeqAndPopulate(false /*isPR */, true /* forwardExpirationDestroy */, 
+        true /* eviction */, false /* evictionOverflow */, 
+        false /* expirationDestroy */, false /* expirationInvalidate */);
+
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, region.size()));
+
+    // Validate events that are not queued.
+    // This guarantees that eviction/expiration is performed and events are
+    // sent all the way to Gateway.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, getEventsNotQueuedSize(aeqId)));
+
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
+    
   }
 
   @Test
-  public void evictionPropogatedUsingIgnoreEvictionAndExpirationAttributeForPR() {
+  public void evictionNotPropogatedUsingForwardExpirationDestroyAttributeForPR() {
     // For PR with eviction-destroy op.
-    // Number of expected events 3. Two for create and One for eviction destroy.
-    createPopulateAndVerifyEvents(true /*isPR */, false /* ignoreEvictionExpiration */, 
-        3 /* expectedEvents */ , true /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, false /* expirationInvalidate */, 
-        true /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    // Number of expected events 3. Two for create and none for eviction destroy.
+    createRegionAeqAndPopulate(true /*isPR */, true /* forwardExpirationDestroy */, 
+        true /* eviction */, false /* evictionOverflow */, 
+        false /* expirationDestroy */, false /* expirationInvalidate */);
+
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, region.size()));
+
+    // Validate events that are not queued.
+    // This guarantees that eviction/expiration is performed and events are
+    // sent all the way to Gateway.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, getEventsNotQueuedSize(aeqId)));
+
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
+
   }
 
   @Test
-  public void overflowNotPropogatedUsingIgnoreEvictionAndExpirationAttribute() {
+  public void overflowNotPropogatedUsingForwardExpirationDestroyAttribute() {
     // For Replicated Region with eviction-overflow op.
     // Number of expected events 2. Two for create and non for eviction overflow.
-    createPopulateAndVerifyEvents(false /*isPR */, false /* ignoreEvictionExpiration */,  
-        2 /* expectedEvents */ , false /* eviction */, true /* evictionOverflow */, 
-        false /* expirationDestroy */, false /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(false /*isPR */, true /* forwardExpirationDestroy */,  
+        false /* eviction */, true /* evictionOverflow */, 
+        false /* expirationDestroy */, false /* expirationInvalidate */);
+    
+    // Wait for region to evict/expire events.
+    LocalRegion lr = (LocalRegion)region;
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, lr.getDiskRegion().getStats().getNumOverflowOnDisk()));
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
   }
 
   @Test
-  public void overflowNotPropogatedUsingIgnoreEvictionAndExpirationAttributeForPR() {
+  public void overflowNotPropogatedUsingForwardExpirationDestroyAttributeForPR() {
     // For PR with eviction-overflow op.
     // Number of expected events 2. Two for create and non for eviction overflow.
-    createPopulateAndVerifyEvents(true /*isPR */, false /* ignoreEvictionExpiration */,  
-        2 /* expectedEvents */ , false /* eviction */, true /* evictionOverflow */, 
-        false /* expirationDestroy */, false /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(true /*isPR */, true /* forwardExpirationDestroy */,  
+        false /* eviction */, true /* evictionOverflow */, 
+        false /* expirationDestroy */, false /* expirationInvalidate */);
+    
+    // Wait for region to evict/expire events.
+    PartitionedRegion pr = (PartitionedRegion)region;
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(1, pr.getDiskRegionStats().getNumOverflowOnDisk()));
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
   }
 
   @Test
-  public void expirationDestroyPropogatedUsingIgnoreEvictionAndExpirationAttribute() {
+  public void expirationDestroyPropogatedUsingForwardExpirationDestroyAttribute() {
     // For Replicated Region with expiration-destroy op.
     // Number of expected events 4. Two for create and Two for expiration destroy.
-    createPopulateAndVerifyEvents(false /*isPR */, false /* ignoreEvictionExpiration */,  
-        4 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        true /* expirationDestroy */, false /* expirationInvalidate */, 
-        true /* checkForDestroyOp */, false /* checkForInvalidateOp */);    
+    createRegionAeqAndPopulate(false /*isPR */, true /* forwardExpirationDestroy */,  
+        false /* eviction */, false /* evictionOverflow */, 
+        true /* expirationDestroy */, false /* expirationInvalidate */); 
+    
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(0, region.size()));
+    
+    Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> assertEquals(4, getEventsReceived(aeqId)));
+    
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> assertEquals(4, events.size()));
+    
+    assertTrue("Expiration event not arrived", checkForOperation(events, false, true));
+        
   }
 
   @Test
-  public void expirationDestroyPropogatedUsingIgnoreEvictionAndExpirationAttributeForPR() {
+  public void expirationDestroyPropogatedUsingForwardExpirationDestroyAttributeForPR() {
     // For PR with expiration-destroy op.
     // Number of expected events 4. Two for create and Two for expiration destroy.
-    createPopulateAndVerifyEvents(true /*isPR */, false /* ignoreEvictionExpiration */,  
-        4 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        true /* expirationDestroy */, false /* expirationInvalidate */, 
-        true /* checkForDestroyOp */, false /* checkForInvalidateOp */);    
+    createRegionAeqAndPopulate(true /*isPR */, true /* forwardExpirationDestroy */,  
+        false /* eviction */, false /* evictionOverflow */, 
+        true /* expirationDestroy */, false /* expirationInvalidate */); 
+    
+    // Wait for region to evict/expire events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(0, region.size()));
+    
+    Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> assertEquals(4, getEventsReceived(aeqId)));
+    
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(120, TimeUnit.SECONDS).until(() -> assertEquals(4, events.size()));
+    
+    assertTrue("Expiration event not arrived", checkForOperation(events, false, true));
+
   }
 
   @Test
-  public void expirationInvalidateNotPropogatedUsingIgnoreEvictionAndExpirationAttribute() {
+  public void expirationInvalidateNotPropogatedUsingForwardExpirationDestroyAttribute() {
     // For Replicated Region with expiration-invalidate op.
     // Currently invalidate event callbacks are not made to GateWay sender.
     // Invalidates are not sent to AEQ.
     // Number of expected events 2. None for expiration invalidate.
-    createPopulateAndVerifyEvents(false /*isPR */, false /* ignoreEvictionExpiration */,  
-        2 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, true /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(false /*isPR */, true /* forwardExpirationDestroy */,
+        false /* eviction */, false /* evictionOverflow */, 
+        false /* expirationDestroy */, true /* expirationInvalidate */);
+    
+ // Wait for region to evict/expire events.
+    LocalRegion lr = (LocalRegion)region;
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, lr.getCachePerfStats().getInvalidates()));
+    
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+    
   }
 
   @Test
-  public void expirationInvalidateNotPropogatedUsingIgnoreEvictionAndExpirationAttributeForPR() {
+  public void expirationInvalidateNotPropogatedUsingForwardExpirationDestroyAttributeForPR() {
     // For PR with expiration-invalidate op.
     // Currently invalidate event callbacks are not made to GateWay sender.
     // Invalidates are not sent to AEQ.
     // Number of expected events 2. None for expiration invalidate.
-    createPopulateAndVerifyEvents(true /*isPR */, false /* ignoreEvictionExpiration */,  
-        2 /* expectedEvents */ , false /* eviction */, false /* evictionOverflow */, 
-        false /* expirationDestroy */, true /* expirationInvalidate */, 
-        false /* checkForDestroyOp */, false /* checkForInvalidateOp */);
+    createRegionAeqAndPopulate(true /*isPR */, true /* forwardExpirationDestroy */,
+        false /* eviction */, false /* evictionOverflow */, 
+        false /* expirationDestroy */, true /* expirationInvalidate */);
+    
+    // Wait for region to evict/expire events.
+    LocalRegion lr = (LocalRegion)region;
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, lr.getCachePerfStats().getInvalidates()));
+    
+    // The AQListner should get expected events.
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> assertEquals(2, events.size()));
+
   }
 
   
-  
-  private void createPopulateAndVerifyEvents(boolean isPR, boolean ignoreEvictionExpiration, int expectedEvents, boolean eviction, boolean evictionOverflow, 
-      boolean expirationDestroy, boolean expirationInvalidate, boolean checkForDestroyOp, boolean checkForInvalidateOp) {
-    
+
+  private void createRegionAeqAndPopulate(boolean isPR, boolean forwardExpirationDestroy, boolean eviction, boolean evictionOverflow, 
+      boolean expirationDestroy, boolean expirationInvalidate) {
     // String aeqId = "AEQTest";
     String aeqId = name.getMethodName();
-    
-    // To store AEQ events for validation.
-    List<AsyncEvent> events = new ArrayList<AsyncEvent>();
-    
+
     // Create AEQ
-    createAsyncEventQueue(aeqId, ignoreEvictionExpiration, events);    
-    
-    // Create region with eviction/expiration
-    Region r = createRegion("ReplicatedRegionForAEQ", isPR, aeqId, eviction, evictionOverflow, expirationDestroy, expirationInvalidate);
-    
-    // Populate region with two entires.
-    r.put("Key-1", "Value-1");
-    r.put("Key-2", "Value-2");
+    createAsyncEventQueue(aeqId, forwardExpirationDestroy, events);
+
+    region = createRegion("ReplicatedRegionForAEQ", isPR, aeqId, eviction, evictionOverflow, expirationDestroy, expirationInvalidate);
     
-    // The AQListner should get two events. One for create, one for destroy.
-    Awaitility.await().atMost(100, TimeUnit.SECONDS).until(() -> {return (events.size() == expectedEvents);});
+    // Populate region with two entries.
+    region.put("Key-1", "Value-1");
+    region.put("Key-2", "Value-2");
     
-    // Check for the expected operation.
-    if (checkForDestroyOp) {
-      assertTrue("Expiration event not arrived", checkForOperation(events, false, true));
-    }
+    try {
+      Thread.sleep(2000);
+    } catch (Exception ex) {}
+  }
 
-    if (checkForInvalidateOp) {
-      assertTrue("Invalidate event not arrived", checkForOperation(events, true, false));
-    }
-    
-    // Test complete. Destroy region.
-    r.destroyRegion();
+
+  private void waitForAEQEventsNotQueued() {
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {return (getEventsNotQueuedSize(aeqId) >= 1);});
   }
+  
 
   private boolean checkForOperation(List<AsyncEvent> events, boolean invalidate, boolean destroy) {
     boolean found = false;
@@ -278,11 +405,24 @@ public class AsyncEventQueueEvictionAndExpirationJUnitTest {
     }
     return found;
   }
+  
+  public int getEventsNotQueuedSize(String aeqId) {
+    AsyncEventQueueImpl aeq  = (AsyncEventQueueImpl)cache.getAsyncEventQueue(aeqId);
+    AbstractGatewaySender sender = (AbstractGatewaySender)aeq.getSender();
+    return sender.getStatistics().getEventsNotQueued();
+  }
+
+
+  public int getEventsReceived(String aeqId) {
+    AsyncEventQueueImpl aeq  = (AsyncEventQueueImpl)cache.getAsyncEventQueue(aeqId);
+    AbstractGatewaySender sender = (AbstractGatewaySender)aeq.getSender();
+    return sender.getStatistics().getEventsReceived();
+  }
 
-  private void createAsyncEventQueue(String id, boolean ignoreEvictionExpiration, List<AsyncEvent> storeEvents) {
+  private void createAsyncEventQueue(String id, boolean forwardExpirationDestroy, List<AsyncEvent> storeEvents) {
     AsyncEventListener al = this.createAsyncListener(storeEvents);
     aeq = cache.createAsyncEventQueueFactory().setParallel(false)
-        .setIgnoreEvictionAndExpiration(ignoreEvictionExpiration)
+        .setForwardExpirationDestroy(forwardExpirationDestroy)
         .setBatchSize(1).setBatchTimeInterval(1).create(id, al);
   }
   
@@ -318,16 +458,16 @@ public class AsyncEventQueueEvictionAndExpirationJUnitTest {
       
       @Override
       public void close() {
-        // TODO Auto-generated method stub
       }
 
       @Override
       public boolean processEvents(List<AsyncEvent> arg0) {
-        System.out.println("AEQ Listener.process()");
-        new Exception("Stack trace for AsyncEventQueue").printStackTrace();
-        // TODO Auto-generated method stub
-        aeList.addAll(arg0);
-        System.out.println("AEQ Event :" + arg0);
+        try {
+          synchronized(aeList) {
+            aeList.add(arg0.get(0));
+          }
+        } catch (Exception ex) {
+        }
         return true;
       }
     };

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXmlGeode10DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXmlGeode10DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXmlGeode10DUnitTest.java
index b5f7f45..6f6e9e9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXmlGeode10DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/CacheXmlGeode10DUnitTest.java
@@ -254,7 +254,7 @@ public class CacheXmlGeode10DUnitTest extends CacheXml81DUnitTest {
 
   @SuppressWarnings("rawtypes")
   @Test
-  public void testAsyncEventQueueIsEnableEvictionAndExpirationAttribute() {
+  public void testAsyncEventQueueIsForwardExpirationDestroyAttribute() {
 
     final String regionName = "testAsyncEventQueueIsEnableEvictionAndExpirationAttribute";
 
@@ -262,22 +262,22 @@ public class CacheXmlGeode10DUnitTest extends CacheXml81DUnitTest {
     final CacheCreation cache = new CacheCreation();
     AsyncEventQueueFactory factory = cache.createAsyncEventQueueFactory();
 
-    
+
     AsyncEventListener listener = new MyAsyncEventListenerGeode10();
 
-    // Test for default ignoreEvictionAndExpiration attribute value (which is true)
-    String aeqId1 = "aeqWithDefaultIgnoreEE";
+    // Test for default forwardExpirationDestroy attribute value (which is false)
+    String aeqId1 = "aeqWithDefaultFED";
     factory.create(aeqId1,listener);
     AsyncEventQueue aeq1 = cache.getAsyncEventQueue(aeqId1);
-    assertTrue(aeq1.isIgnoreEvictionAndExpiration());
+    assertFalse(aeq1.isForwardExpirationDestroy());
 
-    // Test by setting ignoreEvictionAndExpiration attribute value.
-    String aeqId2 = "aeqWithIgnoreEEsetToFalse";
-    factory.setIgnoreEvictionAndExpiration(false);
+    // Test by setting forwardExpirationDestroy attribute value.
+    String aeqId2 = "aeqWithFEDsetToTrue";
+    factory.setForwardExpirationDestroy(true);
     factory.create(aeqId2,listener);
 
     AsyncEventQueue aeq2 = cache.getAsyncEventQueue(aeqId2);
-    assertFalse(aeq2.isIgnoreEvictionAndExpiration());
+    assertTrue(aeq2.isForwardExpirationDestroy());
 
     // Create region and set the AsyncEventQueue
     final RegionAttributesCreation attrs = new RegionAttributesCreation(cache);
@@ -287,17 +287,16 @@ public class CacheXmlGeode10DUnitTest extends CacheXml81DUnitTest {
     assertNotNull(regionBefore);
     assertTrue(regionBefore.getAttributes().getAsyncEventQueueIds().size() == 1);
 
-
     testXml(cache);
 
     final Cache c = getCache();
     assertNotNull(c);
 
     aeq1 = c.getAsyncEventQueue(aeqId1);
-    assertTrue(aeq1.isIgnoreEvictionAndExpiration());
+    assertFalse(aeq1.isForwardExpirationDestroy());
 
     aeq2 = c.getAsyncEventQueue(aeqId2);
-    assertFalse(aeq2.isIgnoreEvictionAndExpiration());
+    assertTrue(aeq2.isForwardExpirationDestroy());
 
     final Region regionAfter = c.getRegion(regionName);
     assertNotNull(regionAfter);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
index 152e55a..86c61b6 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommandsDUnitTest.java
@@ -167,7 +167,7 @@ public class QueueCommandsDUnitTest extends CliCommandTestBase {
     commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER,
         "com.qcdunit.QueueCommandsDUnitTestHelper");
     commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISKSYNCHRONOUS, "false");
-    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__IGNORE_EVICTION_EXPIRATION, "false");
+    commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__FORWARD_EXPIRATION_DESTROY, "true");
     commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER,
         "com.qcdunit.QueueCommandsDUnitTestHelper");
     commandStringBuilder.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE, "param1");
@@ -209,7 +209,7 @@ public class QueueCommandsDUnitTest extends CliCommandTestBase {
         assertEquals(queue.getGatewayEventSubstitutionFilter().getClass().getName(),
             "com.qcdunit.QueueCommandsDUnitTestHelper");
         assertEquals(queue.isDiskSynchronous(), false);
-        assertEquals(queue.isIgnoreEvictionAndExpiration(), false);
+        assertEquals(queue.isForwardExpirationDestroy(), true);
         assertEquals(queue.getAsyncEventListener().getClass().getName(), "com.qcdunit.QueueCommandsDUnitTestHelper");
       }
     });

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/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 2147219..3c85d5d 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
@@ -256,7 +256,6 @@ com/gemstone/gemfire/internal/admin/CompoundRegionSnapshot,true,6295026394298398
 com/gemstone/gemfire/internal/admin/StatAlert,true,5725457607122449170,definitionId:int,time:java/util/Date,values:java/lang/Number[]
 com/gemstone/gemfire/internal/admin/remote/DistributionLocatorId,true,6587390186971937865,bindAddress:java/lang/String,host:java/net/InetAddress,hostnameForClients:java/lang/String,peerLocator:boolean,port:int,serverLocator:boolean
 com/gemstone/gemfire/internal/admin/remote/EntryValueNodeImpl,false,fields:com/gemstone/gemfire/internal/admin/remote/EntryValueNodeImpl[],name:java/lang/String,primitive:boolean,primitiveVal:java/lang/Object,type:java/lang/String
-com/gemstone/gemfire/internal/cache/AbstractRegionMap$1,false,this$0:com/gemstone/gemfire/internal/cache/AbstractRegionMap
 com/gemstone/gemfire/internal/cache/BackupLock,false,backupDone:java/util/concurrent/locks/Condition,backupThread:java/lang/Thread,isBackingUp:boolean
 com/gemstone/gemfire/internal/cache/BucketAdvisor$SetFromMap,true,2454657854757543876,m:java/util/Map
 com/gemstone/gemfire/internal/cache/BucketNotFoundException,false
@@ -281,7 +280,6 @@ com/gemstone/gemfire/internal/cache/ForceableLinkedBlockingQueue,true,-690393397
 com/gemstone/gemfire/internal/cache/GemFireCacheImpl$3,true,1,this$0:com/gemstone/gemfire/internal/cache/GemFireCacheImpl
 com/gemstone/gemfire/internal/cache/GemFireCacheImpl$4,true,1,this$0:com/gemstone/gemfire/internal/cache/GemFireCacheImpl
 com/gemstone/gemfire/internal/cache/GemFireCacheImpl$5,true,1,this$0:com/gemstone/gemfire/internal/cache/GemFireCacheImpl
-com/gemstone/gemfire/internal/cache/GemFireCacheImpl$6,true,1,this$0:com/gemstone/gemfire/internal/cache/GemFireCacheImpl
 com/gemstone/gemfire/internal/cache/IdentityArrayList,true,449125332499184497,size:int,wrapped:boolean
 com/gemstone/gemfire/internal/cache/IncomingGatewayStatus,true,-4579815367602658353,_memberId:java/lang/String,_socketAddress:java/net/InetAddress,_socketPort:int
 com/gemstone/gemfire/internal/cache/InitialImageOperation$GIIStatus,false
@@ -677,7 +675,7 @@ com/gemstone/gemfire/management/internal/cli/exceptions/CliException,false
 com/gemstone/gemfire/management/internal/cli/exceptions/CreateSubregionException,true,4387344870743824916
 com/gemstone/gemfire/management/internal/cli/exceptions/IndexNotFoundException,true,1,indexName:java/lang/String,message:java/lang/String
 com/gemstone/gemfire/management/internal/cli/functions/AlterRuntimeConfigFunction,true,1
-com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs,true,-6524494645663740872,asyncEventQueueId:java/lang/String,batchSize:int,batchTimeInterval:int,diskStoreName:java/lang/String,diskSynchronous:boolean,dispatcherThreads:int,enableBatchConflation:boolean,gatewayEventFilters:java/lang/String[],gatewaySubstitutionFilter:java/lang/String,ignoreEvictionAndExpiration:boolean,isParallel:boolean,listenerClassName:java/lang/String,listenerProperties:java/util/Properties,maxQueueMemory:int,orderPolicy:java/lang/String,persistent:boolean
+com/gemstone/gemfire/management/internal/cli/functions/AsyncEventQueueFunctionArgs,true,-6524494645663740872,asyncEventQueueId:java/lang/String,batchSize:int,batchTimeInterval:int,diskStoreName:java/lang/String,diskSynchronous:boolean,dispatcherThreads:int,enableBatchConflation:boolean,forwardExpirationDestroy:boolean,gatewayEventFilters:java/lang/String[],gatewaySubstitutionFilter:java/lang/String,isParallel:boolean,listenerClassName:java/lang/String,listenerProperties:java/util/Properties,maxQueueMemory:int,orderPolicy:java/lang/String,persistent:boolean
 com/gemstone/gemfire/management/internal/cli/functions/ChangeLogLevelFunction,true,1
 com/gemstone/gemfire/management/internal/cli/functions/CloseDurableClientFunction,true,1
 com/gemstone/gemfire/management/internal/cli/functions/CloseDurableCqFunction,true,1

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/52a13e82/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/cli/commands/golden-help-offline.properties
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/cli/commands/golden-help-offline.properties b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/cli/commands/golden-help-offline.properties
index 3c0d388..0cd7a07 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/cli/commands/golden-help-offline.properties
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/cli/commands/golden-help-offline.properties
@@ -540,7 +540,7 @@ SYNTAX\n\
 \ \ \ \ create async-event-queue --id=value --listener=value [--group=value(nullvalue)*]\n\
 \ \ \ \ [--parallel(=value)?] [--enable-batch-conflation(=value)?] [--batch-size=value]\n\
 \ \ \ \ [--batch-time-interval=value] [--persistent(=value)?] [--disk-store=value]\n\
-\ \ \ \ [--disk-synchronous(=value)?] [--ignore-eviction-expiration(=value)?]\n\
+\ \ \ \ [--disk-synchronous(=value)?] [--forward-expiration-destroy(=value)?]\n\
 \ \ \ \ [--max-queue-memory=value] [--dispatcher-threads=value] [--order-policy=value]\n\
 \ \ \ \ [--gateway-event-filter=value(,value)*] [--gateway-event-substitution-filter=value]\n\
 \ \ \ \ [--listener-param=value(,value)*]\n\
@@ -583,11 +583,11 @@ PARAMETERS\n\
 \ \ \ \ \ \ \ \ Required: false\n\
 \ \ \ \ \ \ \ \ Default (if the parameter is specified without value): true\n\
 \ \ \ \ \ \ \ \ Default (if the parameter is not specified): true\n\
-\ \ \ \ ignore-eviction-expiration\n\
-\ \ \ \ \ \ \ \ Whether to ignore eviction and expiration events.\n\
+\ \ \ \ forward-expiration-destroy\n\
+\ \ \ \ \ \ \ \ Whether to forward expiration destroy events.\n\
 \ \ \ \ \ \ \ \ Required: false\n\
-\ \ \ \ \ \ \ \ Default (if the parameter is specified without value): true\n\
-\ \ \ \ \ \ \ \ Default (if the parameter is not specified): true\n\
+\ \ \ \ \ \ \ \ Default (if the parameter is specified without value): false\n\
+\ \ \ \ \ \ \ \ Default (if the parameter is not specified): false\n\
 \ \ \ \ max-queue-memory\n\
 \ \ \ \ \ \ \ \ Maximum amount of memory, in megabytes, that the queue can consume before overflowing to\n\
 \ \ \ \ \ \ \ \ disk.\n\


[02/16] incubator-geode git commit: GEODE-1374: Run flaky tests in a separate gradle task

Posted by kl...@apache.org.
GEODE-1374: Run flaky tests in a separate gradle task

Add a new test task specifically for running tests annotated with
@FlakyTest.  This allows us to isolate tests that exhibit
intermittent failures.  Other changes include:

- Exclude flaky tests from test, integrationTest, uiTest, and
  distributedTest tasks
- Cleanup how tests are found using the minimum set of
  include/exclude categories


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

Branch: refs/heads/feature/GEODE-1565
Commit: 3dd2efd934c0425fdc7de19f5621340213e1baee
Parents: ca5b45c
Author: Anthony Baker <ab...@apache.org>
Authored: Tue Jun 14 09:51:53 2016 -0700
Committer: Anthony Baker <ab...@apache.org>
Committed: Thu Jun 16 11:58:22 2016 -0700

----------------------------------------------------------------------
 gradle/test.gradle | 63 +++++++++++++++----------------------------------
 1 file changed, 19 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3dd2efd9/gradle/test.gradle
----------------------------------------------------------------------
diff --git a/gradle/test.gradle b/gradle/test.gradle
index 1d3701f..eb10cad 100644
--- a/gradle/test.gradle
+++ b/gradle/test.gradle
@@ -91,56 +91,32 @@ subprojects {
 
 
   test {
-    include '**/*Test.class'
     useJUnit {
       includeCategories 'com.gemstone.gemfire.test.junit.categories.UnitTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.IntegrationTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.DistributedTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.PerformanceTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.HydraTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.ContainerTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.UITest'
+      excludeCategories 'com.gemstone.gemfire.test.junit.categories.FlakyTest'
     }
     
-    // run each test in its own vm to avoid interference issues if a test doesn't clean up
-    // state
-    //forkEvery 1
-    
     doFirst {
       writeTestProperties(buildDir, name)
     }
   }
 
   task integrationTest(type:Test) {
-    include '**/*Test.class'
-    exclude '**/*DUnitTest.class'
     useJUnit {
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.UnitTest'
       includeCategories 'com.gemstone.gemfire.test.junit.categories.IntegrationTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.DistributedTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.PerformanceTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.HydraTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.ContainerTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.UITest'
+      excludeCategories 'com.gemstone.gemfire.test.junit.categories.FlakyTest'
     }
 
     forkEvery 1
-
     doFirst {
       writeTestProperties(buildDir, name)
     }
   }
 
   task uiTest(type:Test) {
-    include '**/*Test.class'
     useJUnit {
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.UnitTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.IntegrationTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.DistributedTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.PerformanceTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.HydraTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.ContainerTest'
       includeCategories 'com.gemstone.gemfire.test.junit.categories.UITest'
+      excludeCategories 'com.gemstone.gemfire.test.junit.categories.FlakyTest'
     }
 
     doFirst {
@@ -149,24 +125,22 @@ subprojects {
   }
   
   task distributedTest(type:Test) {
-    include '**/*Test.class'
-    
-    // maxParallelForks = 2
-    // maxParallelForks = Runtime.runtime.availableProcessors()
-    
     useJUnit {
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.UnitTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.IntegrationTest'
       includeCategories 'com.gemstone.gemfire.test.junit.categories.DistributedTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.PerformanceTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.HydraTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.ContainerTest'
-      excludeCategories 'com.gemstone.gemfire.test.junit.categories.UITest'
+      excludeCategories 'com.gemstone.gemfire.test.junit.categories.FlakyTest'
     }
-    
-    //I'm hoping this might deal with SOME OOMEs I've seen
     forkEvery 30
-
+  }
+  
+  task flakyTest(type:Test) {
+    useJUnit {
+      includeCategories 'com.gemstone.gemfire.test.junit.categories.FlakyTest'
+    }
+    
+    forkEvery 1
+    doFirst {
+      writeTestProperties(buildDir, name)
+    }
   }
 
   // By proving a file with an arbitrary list of test classes, we can select only those
@@ -240,17 +214,18 @@ subprojects {
   })
 
   // Make precheckin task run all validation tests for checking in code.
-  task precheckin (dependsOn: [ build, integrationTest, distributedTest ]) {
-    description 'Run this task before checking in code to validate changes. This task combines the following tasks: build, integrationTest, and distributedTest'
+  task precheckin (dependsOn: [ build, integrationTest, distributedTest, flakyTest ]) {
+    description 'Run this task before checking in code to validate changes. This task combines the following tasks: build, integrationTest, distributedTest, and flakyTest'
   }
 
   check.dependsOn checkMissedTests
   
-  combineReports.mustRunAfter check, test, integrationTest, distributedTest, checkMissedTests
+  combineReports.mustRunAfter check, test, integrationTest, distributedTest, flakyTest, checkMissedTests
   build.finalizedBy combineReports
   check.finalizedBy combineReports
   test.finalizedBy combineReports
   integrationTest.finalizedBy combineReports
   distributedTest.finalizedBy combineReports
+  flakyTest.finalizedBy combineReports
   checkMissedTests.finalizedBy combineReports
 }


[15/16] incubator-geode git commit: GEODE-1493: config/gemfire.properties that is shipped with the geode distribution contains user specific info

Posted by kl...@apache.org.
GEODE-1493: config/gemfire.properties that is shipped with the geode distribution contains user specific info

 * Removed user home directory as default cluster-configuration-dir
 * Now only IPv4 mcast address is used as default mcast address,
   as JGroups can convert IPv4 mcast address to equivalent IPv6.


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

Branch: refs/heads/feature/GEODE-1565
Commit: 832ddd17b9743417421379b8233fbdc45c1109a6
Parents: f606706
Author: Sai Boorlagadda <sb...@pivotal.io>
Authored: Thu Jun 16 20:16:18 2016 -0700
Committer: Sai Boorlagadda <sb...@pivotal.io>
Committed: Fri Jun 17 17:38:04 2016 -0700

----------------------------------------------------------------------
 .../distributed/internal/AbstractDistributionConfig.java    | 9 +++------
 .../distributed/internal/DistributionConfigImpl.java        | 2 +-
 2 files changed, 4 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/832ddd17/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
index c7eca0f..01c893f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/AbstractDistributionConfig.java
@@ -1148,12 +1148,9 @@ public abstract class AbstractDistributionConfig
   }
 
   public static final InetAddress _getDefaultMcastAddress() {
-    String ipLiteral;
-    if ( SocketCreator.preferIPv6Addresses() ) {
-      ipLiteral = "FF38::1234"; // fix for bug 30014
-    } else {
-      ipLiteral = "239.192.81.1"; // fix for bug 30014
-    }
+    //Default MCast address can be just IPv4 address.
+    //On IPv6 machines, JGroups converts IPv4 address to equivalent IPv6 address.
+    String ipLiteral = "239.192.81.1";
     try {
       return InetAddress.getByName(ipLiteral);
     } catch (UnknownHostException ex) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/832ddd17/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java
index b793e34..d31c739 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java
@@ -280,7 +280,7 @@ public class DistributionConfigImpl
   protected boolean enableSharedConfiguration = DistributionConfig.DEFAULT_ENABLE_CLUSTER_CONFIGURATION;
   protected boolean useSharedConfiguration = DistributionConfig.DEFAULT_USE_CLUSTER_CONFIGURATION;
   protected boolean loadSharedConfigurationFromDir = DistributionConfig.DEFAULT_LOAD_CLUSTER_CONFIG_FROM_DIR;
-  protected String clusterConfigDir = DistributionConfig.DEFAULT_CLUSTER_CONFIGURATION_DIR;
+  protected String clusterConfigDir = "";
   
   
   private int httpServicePort = DEFAULT_HTTP_SERVICE_PORT;


[14/16] incubator-geode git commit: GEODE-985: rebrand GFSH help for Apache Geode GEODE-1408: correct alter region help in gfsh

Posted by kl...@apache.org.
GEODE-985: rebrand GFSH help for Apache Geode
GEODE-1408: correct alter region help in gfsh

* This closes #167 [klund@apache.org]


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

Branch: refs/heads/feature/GEODE-1565
Commit: f606706e5f0f311ec734f93beb343db37e512c1b
Parents: fc76608
Author: gmeilen <gr...@gmail.com>
Authored: Fri Jun 17 15:30:20 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Jun 17 15:36:04 2016 -0700

----------------------------------------------------------------------
 .../LauncherLifecycleCommandsDUnitTest.java     |   8 +-
 .../gemfire/management/cli/CliMetaData.java     |   3 +-
 .../internal/cli/commands/ConfigCommands.java   |   6 +-
 .../CreateAlterDestroyRegionCommands.java       |  20 +-
 .../internal/cli/commands/DataCommands.java     |  38 ++--
 .../internal/cli/commands/DeployCommands.java   |   6 +-
 .../cli/commands/DiskStoreCommands.java         |  28 +--
 ...ExportImportSharedConfigurationCommands.java |   4 +-
 .../internal/cli/commands/FunctionCommands.java |   6 +-
 .../internal/cli/commands/GfshHelpCommands.java |   4 +-
 .../internal/cli/commands/IndexCommands.java    |  12 +-
 .../cli/commands/LauncherLifecycleCommands.java |  40 ++--
 .../internal/cli/commands/MemberCommands.java   |   4 +-
 .../cli/commands/MiscellaneousCommands.java     |  16 +-
 .../internal/cli/commands/PDXCommands.java      |   6 +-
 .../internal/cli/commands/RegionCommands.java   |   4 +-
 .../internal/cli/commands/ShellCommands.java    |  14 +-
 .../internal/cli/commands/StatusCommands.java   |   2 +-
 .../internal/cli/commands/WanCommands.java      |  24 +--
 .../management/internal/cli/help/CliTopic.java  |  32 +--
 .../internal/cli/i18n/CliStrings.java           | 216 +++++++++----------
 .../internal/cli/shell/JmxOperationInvoker.java |   2 +-
 22 files changed, 250 insertions(+), 245 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
index da4c4ab..1d91834 100644
--- a/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
+++ b/geode-assembly/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommandsDUnitTest.java
@@ -365,7 +365,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
     String resultString = toString(result);
 
     assertTrue(resultString, resultString.contains(
-        MessageFormat.format(CliStrings.GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, StringUtils.EMPTY_STRING,
+        MessageFormat.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, StringUtils.EMPTY_STRING,
             gemfirePropertiesPathname)));
   }
 
@@ -391,7 +391,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
     String resultString = toString(result);
 
     assertTrue(resultString, resultString.contains(
-        MessageFormat.format(CliStrings.GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, "Security ",
+        MessageFormat.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, "Security ",
             gemfireSecurityPropertiesPathname)));
   }
 
@@ -432,7 +432,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
     String resultString = toString(result);
 
     assertTrue(resultString, resultString.contains(
-        MessageFormat.format(CliStrings.GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, StringUtils.EMPTY_STRING,
+        MessageFormat.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, StringUtils.EMPTY_STRING,
             gemfirePropertiesFile)));
   }
 
@@ -453,7 +453,7 @@ public class LauncherLifecycleCommandsDUnitTest extends CliCommandTestBase {
     String resultString = toString(result);
 
     assertTrue(resultString, resultString.contains(
-        MessageFormat.format(CliStrings.GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, "Security ",
+        MessageFormat.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, "Security ",
             gemfireSecuritiesPropertiesFile)));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/cli/CliMetaData.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/cli/CliMetaData.java b/geode-core/src/main/java/com/gemstone/gemfire/management/cli/CliMetaData.java
index a5ed7c5..607f545 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/cli/CliMetaData.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/cli/CliMetaData.java
@@ -22,7 +22,6 @@ import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
 
 import com.gemstone.gemfire.management.internal.cli.CliAroundInterceptor;
-import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 
 /**
  * An annotation to define additional meta-data for commands.
@@ -54,7 +53,7 @@ public @interface CliMetaData {
   boolean writesToSharedConfiguration() default false;
   
   /** In help, topics that are related to this command **/
-  String[] relatedTopic() default com.gemstone.gemfire.management.internal.cli.i18n.CliStrings.DEFAULT_TOPIC_GEMFIRE;
+  String[] relatedTopic() default com.gemstone.gemfire.management.internal.cli.i18n.CliStrings.DEFAULT_TOPIC_GEODE;
 
   /**
    * The fully qualified name of a class which implements the

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommands.java
index de17d1c..d16414c 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ConfigCommands.java
@@ -68,7 +68,7 @@ public class ConfigCommands implements CommandMarker {
   }
 
   @CliCommand(value = { CliStrings.DESCRIBE_CONFIG }, help = CliStrings.DESCRIBE_CONFIG__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = {CliStrings.TOPIC_GEMFIRE_CONFIG})
+  @CliMetaData(shellOnly = false, relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG })
   @ResourceOperation(resource = Resource.CLUSTER, operation= OperationCode.READ)
   public Result describeConfig(
       @CliOption (key = CliStrings.DESCRIBE_CONFIG__MEMBER,
@@ -182,7 +182,7 @@ public class ConfigCommands implements CommandMarker {
    * @return Results of the attempt to write the configuration
    */
   @CliCommand(value = { CliStrings.EXPORT_CONFIG }, help = CliStrings.EXPORT_CONFIG__HELP)
-  @CliMetaData(interceptor = "com.gemstone.gemfire.management.internal.cli.commands.ConfigCommands$Interceptor", relatedTopic = {CliStrings.TOPIC_GEMFIRE_CONFIG})
+  @CliMetaData(interceptor = "com.gemstone.gemfire.management.internal.cli.commands.ConfigCommands$Interceptor", relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result exportConfig(
       @CliOption(key = { CliStrings.EXPORT_CONFIG__MEMBER },
@@ -238,7 +238,7 @@ public class ConfigCommands implements CommandMarker {
 
 
   @CliCommand(value = { CliStrings.ALTER_RUNTIME_CONFIG }, help = CliStrings.ALTER_RUNTIME_CONFIG__HELP)
-  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEMFIRE_CONFIG})
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.MANAGE)
   public Result alterRuntimeConfig(
       @CliOption (key = {CliStrings.ALTER_RUNTIME_CONFIG__MEMBER},

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
index d27786f..741da25 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
@@ -108,7 +108,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand (value = CliStrings.CREATE_REGION, help = CliStrings.CREATE_REGION__HELP)
-  @CliMetaData (relatedTopic = CliStrings.TOPIC_GEMFIRE_REGION, writesToSharedConfiguration = true)
+  @CliMetaData (relatedTopic = CliStrings.TOPIC_GEODE_REGION, writesToSharedConfiguration = true)
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public Result createRegion(
       @CliOption (key = CliStrings.CREATE_REGION__REGION,
@@ -437,7 +437,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
   }  
   
   @CliCommand (value = CliStrings.ALTER_REGION, help = CliStrings.ALTER_REGION__HELP)
-  @CliMetaData (relatedTopic = CliStrings.TOPIC_GEMFIRE_REGION, writesToSharedConfiguration = true)
+  @CliMetaData (relatedTopic = CliStrings.TOPIC_GEODE_REGION, writesToSharedConfiguration = true)
   public Result alterRegion(
       @CliOption (key = CliStrings.ALTER_REGION__REGION,
                   mandatory = true,
@@ -457,7 +457,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
       Integer entryExpirationIdleTime,
       @CliOption (key = CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIMEACTION,
                   unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-                  specifiedDefaultValue = CliMetaData.ANNOTATION_DEFAULT_VALUE,
+                  specifiedDefaultValue = "INVALIDATE",
                   help = CliStrings.ALTER_REGION__ENTRYEXPIRATIONIDLETIMEACTION__HELP)
       String entryExpirationIdleTimeAction,
       @CliOption (key = CliStrings.ALTER_REGION__ENTRYEXPIRATIONTIMETOLIVE,
@@ -467,7 +467,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
       Integer entryExpirationTTL,
       @CliOption (key = CliStrings.ALTER_REGION__ENTRYEXPIRATIONTTLACTION,
                   unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-                  specifiedDefaultValue = CliMetaData.ANNOTATION_DEFAULT_VALUE,
+                  specifiedDefaultValue = "INVALIDATE",
                   help = CliStrings.ALTER_REGION__ENTRYEXPIRATIONTTLACTION__HELP)
       String entryExpirationTTLAction,
       @CliOption (key = CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIME,
@@ -477,7 +477,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
       Integer regionExpirationIdleTime, 
       @CliOption (key = CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIMEACTION,
                   unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-                  specifiedDefaultValue = CliMetaData.ANNOTATION_DEFAULT_VALUE,
+                  specifiedDefaultValue = "INVALIDATE",
                   help = CliStrings.ALTER_REGION__REGIONEXPIRATIONIDLETIMEACTION__HELP)
       String regionExpirationIdleTimeAction,
       @CliOption (key = CliStrings.ALTER_REGION__REGIONEXPIRATIONTTL,
@@ -487,7 +487,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
       Integer regionExpirationTTL, 
       @CliOption (key = CliStrings.ALTER_REGION__REGIONEXPIRATIONTTLACTION,
                   unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-                  specifiedDefaultValue = CliMetaData.ANNOTATION_DEFAULT_VALUE,
+                  specifiedDefaultValue = "INVALIDATE",
                   help = CliStrings.ALTER_REGION__REGIONEXPIRATIONTTLACTION__HELP)
       String regionExpirationTTLAction,          
       @CliOption (key = CliStrings.ALTER_REGION__CACHELISTENER,
@@ -498,12 +498,12 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
       String[] cacheListeners,
       @CliOption (key = CliStrings.ALTER_REGION__CACHELOADER,
                   unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-                  specifiedDefaultValue = CliMetaData.ANNOTATION_DEFAULT_VALUE,
+                  specifiedDefaultValue = "help",
                   help = CliStrings.ALTER_REGION__CACHELOADER__HELP)
       String cacheLoader,
       @CliOption (key = CliStrings.ALTER_REGION__CACHEWRITER,
                   unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-                  specifiedDefaultValue = CliMetaData.ANNOTATION_DEFAULT_VALUE,
+                  specifiedDefaultValue = "null",
                   help = CliStrings.ALTER_REGION__CACHEWRITER__HELP)
       String cacheWriter,
       @CliOption (key = CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID,
@@ -997,7 +997,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = { CliStrings.DESTROY_REGION }, help = CliStrings.DESTROY_REGION__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = CliStrings.TOPIC_GEMFIRE_REGION, writesToSharedConfiguration = true)
+  @CliMetaData(shellOnly = false, relatedTopic = CliStrings.TOPIC_GEODE_REGION, writesToSharedConfiguration = true)
   @ResourceOperation(resource=Resource.DATA, operation = OperationCode.MANAGE)
   public Result destroyRegion(
       @CliOption(key = CliStrings.DESTROY_REGION__REGION,
@@ -1031,7 +1031,7 @@ public class CreateAlterDestroyRegionCommands extends AbstractCommandsSupport {
 
       if (regionMembersList.size() == 0) {
         return ResultBuilder.createUserErrorResult(
-            CliStrings.format(CliStrings.DESTROY_REGION__MSG__COULDNOT_FIND_REGIONPATH_0_IN_GEMFIRE,
+            CliStrings.format(CliStrings.DESTROY_REGION__MSG__COULDNOT_FIND_REGIONPATH_0_IN_GEODE,
                 new Object[]{regionPath, "jmx-manager-update-rate milliseconds"}));
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
index 8d93344..bbf2b02 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommands.java
@@ -95,8 +95,9 @@ public class DataCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.REBALANCE, help = CliStrings.REBALANCE__HELP)
-  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEMFIRE_DATA,
-      CliStrings.TOPIC_GEMFIRE_REGION })
+  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEODE_DATA,
+      CliStrings.TOPIC_GEODE_REGION
+  })
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public Result rebalance(
       @CliOption(key = CliStrings.REBALANCE__INCLUDEREGION, unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, help = CliStrings.REBALANCE__INCLUDEREGION__HELP) String[] includeRegions,
@@ -833,8 +834,9 @@ public class DataCommands implements CommandMarker {
   
 
   @CliCommand(value = CliStrings.EXPORT_DATA, help = CliStrings.EXPORT_DATA__HELP)
-  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEMFIRE_DATA,
-      CliStrings.TOPIC_GEMFIRE_REGION })
+  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEODE_DATA,
+      CliStrings.TOPIC_GEODE_REGION
+  })
   public Result exportData(
       @CliOption(key = CliStrings.EXPORT_DATA__REGION, mandatory = true, optionContext = ConverterHint.REGIONPATH, help = CliStrings.EXPORT_DATA__REGION__HELP) String regionName,
       @CliOption(key = CliStrings.EXPORT_DATA__FILE, unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, mandatory = true, help = CliStrings.EXPORT_DATA__FILE__HELP) String filePath,
@@ -846,10 +848,10 @@ public class DataCommands implements CommandMarker {
         .getDistributedMemberByNameOrId(memberNameOrId);
     Result result = null;
 
-    if (!filePath.endsWith(CliStrings.GEMFIRE_DATA_FILE_EXTENSION)) {
+    if (!filePath.endsWith(CliStrings.GEODE_DATA_FILE_EXTENSION)) {
       return ResultBuilder.createUserErrorResult(CliStrings.format(
           CliStrings.INVALID_FILE_EXTENTION,
-          CliStrings.GEMFIRE_DATA_FILE_EXTENSION));
+          CliStrings.GEODE_DATA_FILE_EXTENSION));
     }
     try {
       if (targetMember != null) {
@@ -889,8 +891,9 @@ public class DataCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.IMPORT_DATA, help = CliStrings.IMPORT_DATA__HELP)
-  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEMFIRE_DATA,
-      CliStrings.TOPIC_GEMFIRE_REGION })
+  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEODE_DATA,
+      CliStrings.TOPIC_GEODE_REGION
+  })
   public Result importData(
       @CliOption(key = CliStrings.IMPORT_DATA__REGION, optionContext = ConverterHint.REGIONPATH, mandatory = true, help = CliStrings.IMPORT_DATA__REGION__HELP) String regionName,
       @CliOption(key = CliStrings.IMPORT_DATA__FILE, mandatory = true, unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, help = CliStrings.IMPORT_DATA__FILE__HELP) String filePath,
@@ -905,10 +908,10 @@ public class DataCommands implements CommandMarker {
       final DistributedMember targetMember = CliUtil
           .getDistributedMemberByNameOrId(memberNameOrId);
 
-      if (!filePath.endsWith(CliStrings.GEMFIRE_DATA_FILE_EXTENSION)) {
+      if (!filePath.endsWith(CliStrings.GEODE_DATA_FILE_EXTENSION)) {
         return ResultBuilder.createUserErrorResult(CliStrings.format(
             CliStrings.INVALID_FILE_EXTENTION,
-            CliStrings.GEMFIRE_DATA_FILE_EXTENSION));
+            CliStrings.GEODE_DATA_FILE_EXTENSION));
       }
       if (targetMember != null) {
         final String args[] = { regionName, filePath };
@@ -946,7 +949,8 @@ public class DataCommands implements CommandMarker {
   }
 
   @CliMetaData(shellOnly = false, relatedTopic = {
-      CliStrings.TOPIC_GEMFIRE_DATA, CliStrings.TOPIC_GEMFIRE_REGION })
+      CliStrings.TOPIC_GEODE_DATA, CliStrings.TOPIC_GEODE_REGION
+  })
   @CliCommand(value = { CliStrings.PUT }, help = CliStrings.PUT__HELP)
   public Result put(
       @CliOption(key = { CliStrings.PUT__KEY }, mandatory = true, help = CliStrings.PUT__KEY__HELP) String key,
@@ -1014,7 +1018,8 @@ public class DataCommands implements CommandMarker {
   }
 
   @CliMetaData(shellOnly = false, relatedTopic = {
-      CliStrings.TOPIC_GEMFIRE_DATA, CliStrings.TOPIC_GEMFIRE_REGION })
+      CliStrings.TOPIC_GEODE_DATA, CliStrings.TOPIC_GEODE_REGION
+  })
   @CliCommand(value = { CliStrings.GET }, help = CliStrings.GET__HELP)
   public Result get(
       @CliOption(key = { CliStrings.GET__KEY }, mandatory = true, help = CliStrings.GET__KEY__HELP) String key,
@@ -1069,7 +1074,8 @@ public class DataCommands implements CommandMarker {
   }
 
   @CliMetaData(shellOnly = false, relatedTopic = {
-      CliStrings.TOPIC_GEMFIRE_DATA, CliStrings.TOPIC_GEMFIRE_REGION })
+      CliStrings.TOPIC_GEODE_DATA, CliStrings.TOPIC_GEODE_REGION
+  })
   @CliCommand(value = { CliStrings.LOCATE_ENTRY }, help = CliStrings.LOCATE_ENTRY__HELP)
   public Result locateEntry(
       @CliOption(key = { CliStrings.LOCATE_ENTRY__KEY }, mandatory = true, help = CliStrings.LOCATE_ENTRY__KEY__HELP) String key,
@@ -1117,7 +1123,8 @@ public class DataCommands implements CommandMarker {
   }
 
   @CliMetaData(shellOnly = false, relatedTopic = {
-      CliStrings.TOPIC_GEMFIRE_DATA, CliStrings.TOPIC_GEMFIRE_REGION })
+      CliStrings.TOPIC_GEODE_DATA, CliStrings.TOPIC_GEODE_REGION
+  })
   @CliCommand(value = { CliStrings.REMOVE }, help = CliStrings.REMOVE__HELP)
   @ResourceOperation(resource=Resource.DATA, operation = OperationCode.MANAGE)
   public Result remove(
@@ -1170,7 +1177,8 @@ public class DataCommands implements CommandMarker {
   }
 
   @CliMetaData(shellOnly = false, relatedTopic = {
-      CliStrings.TOPIC_GEMFIRE_DATA, CliStrings.TOPIC_GEMFIRE_REGION })
+      CliStrings.TOPIC_GEODE_DATA, CliStrings.TOPIC_GEODE_REGION
+  })
   @MultiStepCommand
   @CliCommand(value = { CliStrings.QUERY }, help = CliStrings.QUERY__HELP)
   public Object query(

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommands.java
index 7729dfb..e320383 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DeployCommands.java
@@ -77,7 +77,7 @@ public final class DeployCommands extends AbstractCommandsSupport implements Com
    * @return The result of the attempt to deploy
    */
   @CliCommand(value = { CliStrings.DEPLOY }, help = CliStrings.DEPLOY__HELP)
-  @CliMetaData(interceptor = "com.gemstone.gemfire.management.internal.cli.commands.DeployCommands$Interceptor", relatedTopic={CliStrings.TOPIC_GEMFIRE_CONFIG}, writesToSharedConfiguration=true)
+  @CliMetaData(interceptor = "com.gemstone.gemfire.management.internal.cli.commands.DeployCommands$Interceptor", relatedTopic={CliStrings.TOPIC_GEODE_CONFIG }, writesToSharedConfiguration=true)
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public final Result deploy(
     @CliOption(key = { CliStrings.DEPLOY__GROUP }, help = CliStrings.DEPLOY__GROUP__HELP, optionContext=ConverterHint.MEMBERGROUP)
@@ -160,7 +160,7 @@ public final class DeployCommands extends AbstractCommandsSupport implements Com
    * @return The result of the attempt to undeploy
    */
   @CliCommand(value = { CliStrings.UNDEPLOY }, help = CliStrings.UNDEPLOY__HELP)
-  @CliMetaData(relatedTopic={CliStrings.TOPIC_GEMFIRE_CONFIG}, writesToSharedConfiguration=true)
+  @CliMetaData(relatedTopic={CliStrings.TOPIC_GEODE_CONFIG }, writesToSharedConfiguration=true)
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public final Result undeploy(
       @CliOption(key = { CliStrings.UNDEPLOY__GROUP },
@@ -231,7 +231,7 @@ public final class DeployCommands extends AbstractCommandsSupport implements Com
    * @return List of deployed JAR files
    */
   @CliCommand(value = { CliStrings.LIST_DEPLOYED }, help = CliStrings.LIST_DEPLOYED__HELP)
-  @CliMetaData(relatedTopic={CliStrings.TOPIC_GEMFIRE_CONFIG})
+  @CliMetaData(relatedTopic={CliStrings.TOPIC_GEODE_CONFIG })
   @ResourceOperation(resource = Resource.CLUSTER, operation= OperationCode.READ)
   public final Result listDeployed(
       @CliOption(key = { CliStrings.LIST_DEPLOYED__GROUP },

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommands.java
index 36eda43..eba9d3b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/DiskStoreCommands.java
@@ -119,7 +119,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
   }
   
   @CliCommand(value=CliStrings.BACKUP_DISK_STORE, help=CliStrings.BACKUP_DISK_STORE__HELP)
-  @CliMetaData(relatedTopic={ CliStrings.TOPIC_GEMFIRE_DISKSTORE })
+  @CliMetaData(relatedTopic={ CliStrings.TOPIC_GEODE_DISKSTORE })
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.READ)
   public Result backupDiskStore(
   
@@ -214,7 +214,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
 
 
   @CliCommand(value = CliStrings.LIST_DISK_STORE, help = CliStrings.LIST_DISK_STORE__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEMFIRE_DISKSTORE })
+  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEODE_DISKSTORE })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result listDiskStore() {
     try {
@@ -283,7 +283,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value=CliStrings.CREATE_DISK_STORE, help=CliStrings.CREATE_DISK_STORE__HELP)
-  @CliMetaData(shellOnly=false, relatedTopic={CliStrings.TOPIC_GEMFIRE_DISKSTORE}, writesToSharedConfiguration=true)
+  @CliMetaData(shellOnly=false, relatedTopic={CliStrings.TOPIC_GEODE_DISKSTORE }, writesToSharedConfiguration=true)
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public Result createDiskStore(@CliOption(key=CliStrings.CREATE_DISK_STORE__NAME,
                                            mandatory=true,
@@ -424,7 +424,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
 
     
   @CliCommand(value=CliStrings.COMPACT_DISK_STORE, help=CliStrings.COMPACT_DISK_STORE__HELP)
-  @CliMetaData(shellOnly=false, relatedTopic={CliStrings.TOPIC_GEMFIRE_DISKSTORE})
+  @CliMetaData(shellOnly=false, relatedTopic={CliStrings.TOPIC_GEODE_DISKSTORE })
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public Result compactDiskStore(@CliOption(key=CliStrings.COMPACT_DISK_STORE__NAME,
                                             mandatory=true,
@@ -554,7 +554,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value=CliStrings.COMPACT_OFFLINE_DISK_STORE, help=CliStrings.COMPACT_OFFLINE_DISK_STORE__HELP)
-  @CliMetaData(shellOnly=true, relatedTopic={CliStrings.TOPIC_GEMFIRE_DISKSTORE})
+  @CliMetaData(shellOnly=true, relatedTopic={CliStrings.TOPIC_GEODE_DISKSTORE })
   public Result compactOfflineDiskStore(
                  @CliOption(key=CliStrings.COMPACT_OFFLINE_DISK_STORE__NAME,
                             mandatory=true,
@@ -697,7 +697,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
 
   
   @CliCommand(value=CliStrings.UPGRADE_OFFLINE_DISK_STORE, help=CliStrings.UPGRADE_OFFLINE_DISK_STORE__HELP)
-  @CliMetaData(shellOnly=true, relatedTopic={CliStrings.TOPIC_GEMFIRE_DISKSTORE})
+  @CliMetaData(shellOnly=true, relatedTopic={CliStrings.TOPIC_GEODE_DISKSTORE })
   public Result upgradeOfflineDiskStore(
       @CliOption(key=CliStrings.UPGRADE_OFFLINE_DISK_STORE__NAME, 
       mandatory=true,
@@ -866,7 +866,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.DESCRIBE_DISK_STORE, help = CliStrings.DESCRIBE_DISK_STORE__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEMFIRE_DISKSTORE })
+  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEODE_DISKSTORE })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result describeDiskStore(@CliOption(key = CliStrings.DESCRIBE_DISK_STORE__MEMBER, mandatory = true, optionContext = ConverterHint.MEMBERIDNAME, help = CliStrings.DESCRIBE_DISK_STORE__MEMBER__HELP)
                                   final String memberName,
@@ -993,7 +993,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.REVOKE_MISSING_DISK_STORE, help = CliStrings.REVOKE_MISSING_DISK_STORE__HELP)
-  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEMFIRE_DISKSTORE })
+  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEODE_DISKSTORE })
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public Result revokeMissingDiskStore(
       @CliOption(key = CliStrings.REVOKE_MISSING_DISK_STORE__ID, mandatory = true, help = CliStrings.REVOKE_MISSING_DISK_STORE__ID__HELP)
@@ -1020,7 +1020,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.SHOW_MISSING_DISK_STORE, help = CliStrings.SHOW_MISSING_DISK_STORE__HELP)
-  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEMFIRE_DISKSTORE })
+  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEODE_DISKSTORE })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result showMissingDiskStore() {
 
@@ -1059,7 +1059,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
   
   
   @CliCommand(value=CliStrings.DESCRIBE_OFFLINE_DISK_STORE, help=CliStrings.DESCRIBE_OFFLINE_DISK_STORE__HELP)
-  @CliMetaData(shellOnly=true, relatedTopic={CliStrings.TOPIC_GEMFIRE_DISKSTORE})
+  @CliMetaData(shellOnly=true, relatedTopic={CliStrings.TOPIC_GEODE_DISKSTORE })
   public Result describeOfflineDiskStore(
       @CliOption (key=CliStrings.DESCRIBE_OFFLINE_DISK_STORE__DISKSTORENAME, 
           mandatory=true,
@@ -1107,7 +1107,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
   }
   
   @CliCommand(value=CliStrings.EXPORT_OFFLINE_DISK_STORE, help=CliStrings.EXPORT_OFFLINE_DISK_STORE__HELP)
-  @CliMetaData(shellOnly=true, relatedTopic={CliStrings.TOPIC_GEMFIRE_DISKSTORE})
+  @CliMetaData(shellOnly=true, relatedTopic={CliStrings.TOPIC_GEODE_DISKSTORE })
   public Result exportOfflineDiskStore(
       @CliOption (key=CliStrings.EXPORT_OFFLINE_DISK_STORE__DISKSTORENAME, 
           mandatory=true,
@@ -1155,7 +1155,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value=CliStrings.VALIDATE_DISK_STORE, help=CliStrings.VALIDATE_DISK_STORE__HELP)
-  @CliMetaData(shellOnly=true, relatedTopic = {CliStrings.TOPIC_GEMFIRE_DISKSTORE}) //offline command
+  @CliMetaData(shellOnly=true, relatedTopic = {CliStrings.TOPIC_GEODE_DISKSTORE }) //offline command
   public Result validateDiskStore(
       @CliOption(key=CliStrings.VALIDATE_DISK_STORE__NAME, mandatory=true,
                   help=CliStrings.VALIDATE_DISK_STORE__NAME__HELP)
@@ -1236,7 +1236,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
   
   
   @CliCommand(value=CliStrings.ALTER_DISK_STORE, help=CliStrings.ALTER_DISK_STORE__HELP)
-  @CliMetaData(shellOnly=true, relatedTopic={CliStrings.TOPIC_GEMFIRE_DISKSTORE})
+  @CliMetaData(shellOnly=true, relatedTopic={CliStrings.TOPIC_GEODE_DISKSTORE })
   public Result alterOfflineDiskStore(
       @CliOption  (key=CliStrings.ALTER_DISK_STORE__DISKSTORENAME, 
       mandatory=true,
@@ -1367,7 +1367,7 @@ public class DiskStoreCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value=CliStrings.DESTROY_DISK_STORE, help=CliStrings.DESTROY_DISK_STORE__HELP)
-  @CliMetaData(shellOnly=false, relatedTopic={CliStrings.TOPIC_GEMFIRE_DISKSTORE}, writesToSharedConfiguration=true)
+  @CliMetaData(shellOnly=false, relatedTopic={CliStrings.TOPIC_GEODE_DISKSTORE }, writesToSharedConfiguration=true)
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public Result destroyDiskStore(
       @CliOption  (key=CliStrings.DESTROY_DISK_STORE__NAME, 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ExportImportSharedConfigurationCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ExportImportSharedConfigurationCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ExportImportSharedConfigurationCommands.java
index 085dc19..b830fff 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ExportImportSharedConfigurationCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ExportImportSharedConfigurationCommands.java
@@ -63,7 +63,7 @@ public class ExportImportSharedConfigurationCommands extends AbstractCommandsSup
   private final LoadSharedConfigurationFunction loadSharedConfiguration = new LoadSharedConfigurationFunction();
 
   @CliCommand(value = { CliStrings.EXPORT_SHARED_CONFIG }, help = CliStrings.EXPORT_SHARED_CONFIG__HELP)
-  @CliMetaData(interceptor = "com.gemstone.gemfire.management.internal.cli.commands.ExportImportSharedConfigurationCommands$ExportInterceptor",  readsSharedConfiguration=true, relatedTopic = {CliStrings.TOPIC_GEMFIRE_CONFIG})
+  @CliMetaData(interceptor = "com.gemstone.gemfire.management.internal.cli.commands.ExportImportSharedConfigurationCommands$ExportInterceptor",  readsSharedConfiguration=true, relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result exportSharedConfig(
       @CliOption(key = { CliStrings.EXPORT_SHARED_CONFIG__FILE}, 
@@ -114,7 +114,7 @@ public class ExportImportSharedConfigurationCommands extends AbstractCommandsSup
   }
 
   @CliCommand(value = { CliStrings.IMPORT_SHARED_CONFIG }, help = CliStrings.IMPORT_SHARED_CONFIG__HELP)
-  @CliMetaData(interceptor = "com.gemstone.gemfire.management.internal.cli.commands.ExportImportSharedConfigurationCommands$ImportInterceptor", writesToSharedConfiguration=true, relatedTopic = {CliStrings.TOPIC_GEMFIRE_CONFIG})
+  @CliMetaData(interceptor = "com.gemstone.gemfire.management.internal.cli.commands.ExportImportSharedConfigurationCommands$ImportInterceptor", writesToSharedConfiguration=true, relatedTopic = {CliStrings.TOPIC_GEODE_CONFIG })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.MANAGE)
   @SuppressWarnings("unchecked")
   public Result importSharedConfig(

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommands.java
index 5469e22..ae3d0aa 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/FunctionCommands.java
@@ -81,7 +81,7 @@ public class FunctionCommands implements CommandMarker {
   }
   
   @CliCommand(value = CliStrings.EXECUTE_FUNCTION, help = CliStrings.EXECUTE_FUNCTION__HELP)
-  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEMFIRE_FUNCTION })
+  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEODE_FUNCTION })
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.WRITE)
   public Result executeFunction(
       //TODO: Add optioncontext for functionID
@@ -453,7 +453,7 @@ public class FunctionCommands implements CommandMarker {
   }
   
   @CliCommand(value = CliStrings.DESTROY_FUNCTION, help = CliStrings.DESTROY_FUNCTION__HELP)
-  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEMFIRE_FUNCTION } ,
+  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEODE_FUNCTION } ,
       interceptor = "com.gemstone.gemfire.management.internal.cli.commands.FunctionCommands$Interceptor")  
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   //TODO: Add optioncontext for functionId
@@ -578,7 +578,7 @@ public class FunctionCommands implements CommandMarker {
   }     
   
   @CliCommand(value = CliStrings.LIST_FUNCTION, help = CliStrings.LIST_FUNCTION__HELP)
-  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEMFIRE_FUNCTION })
+  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEODE_FUNCTION })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result listFunction(
       @CliOption(key = CliStrings.LIST_FUNCTION__MATCHES, 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/GfshHelpCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/GfshHelpCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/GfshHelpCommands.java
index ac961cb..932641b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/GfshHelpCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/GfshHelpCommands.java
@@ -47,7 +47,7 @@ public class GfshHelpCommands implements CommandMarker{
   }
   
   @CliCommand(value = CliStrings.HELP, help = CliStrings.HELP__HELP)
-  @CliMetaData(shellOnly=true, relatedTopic = {CliStrings.TOPIC_GEMFIRE_HELP})
+  @CliMetaData(shellOnly=true, relatedTopic = {CliStrings.TOPIC_GEODE_HELP })
   public Result obtainHelp(
       @CliArgument(name = CliStrings.HELP__COMMAND, 
                  argumentContext = CliStrings.PARAM_CONTEXT_HELP, 
@@ -59,7 +59,7 @@ public class GfshHelpCommands implements CommandMarker{
   
   
   @CliCommand(value = CliStrings.HINT, help = CliStrings.HINT__HELP)
-  @CliMetaData(shellOnly=true, relatedTopic = {CliStrings.TOPIC_GEMFIRE_HELP})
+  @CliMetaData(shellOnly=true, relatedTopic = {CliStrings.TOPIC_GEODE_HELP })
   public Result hint(
       @CliArgument(name = CliStrings.HINT__TOPICNAME, 
                 argumentContext = ConverterHint.HINTTOPIC, 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java
index d2ca5f7..dde31a6 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/IndexCommands.java
@@ -87,7 +87,7 @@ public class IndexCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.LIST_INDEX, help = CliStrings.LIST_INDEX__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEMFIRE_REGION, CliStrings.TOPIC_GEMFIRE_DATA})
+  @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEODE_REGION, CliStrings.TOPIC_GEODE_DATA })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result listIndex(@CliOption(key = CliStrings.LIST_INDEX__STATS,
                                      mandatory = false,
@@ -171,7 +171,7 @@ public class IndexCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.CREATE_INDEX, help = CliStrings.CREATE_INDEX__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEMFIRE_REGION, CliStrings.TOPIC_GEMFIRE_DATA}, writesToSharedConfiguration=true)
+  @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEODE_REGION, CliStrings.TOPIC_GEODE_DATA }, writesToSharedConfiguration=true)
   //TODO : Add optionContext for indexName
   public Result createIndex(
       @CliOption (key = CliStrings.CREATE_INDEX__NAME,
@@ -321,7 +321,7 @@ public class IndexCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.DESTROY_INDEX, help = CliStrings.DESTROY_INDEX__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEMFIRE_REGION, CliStrings.TOPIC_GEMFIRE_DATA}, writesToSharedConfiguration=true)
+  @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEODE_REGION, CliStrings.TOPIC_GEODE_DATA }, writesToSharedConfiguration=true)
   public Result destroyIndex(
       @CliOption(
       key = CliStrings.DESTROY_INDEX__NAME,
@@ -463,7 +463,7 @@ public class IndexCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.DEFINE_INDEX, help = CliStrings.DEFINE_INDEX__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEMFIRE_REGION, CliStrings.TOPIC_GEMFIRE_DATA}, writesToSharedConfiguration=true)
+  @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEODE_REGION, CliStrings.TOPIC_GEODE_DATA }, writesToSharedConfiguration=true)
   //TODO : Add optionContext for indexName
   public Result defineIndex(
       @CliOption (key = CliStrings.DEFINE_INDEX_NAME,
@@ -533,7 +533,7 @@ public class IndexCommands extends AbstractCommandsSupport {
   }
   
   @CliCommand(value = CliStrings.CREATE_DEFINED_INDEXES, help = CliStrings.CREATE_DEFINED__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEMFIRE_REGION, CliStrings.TOPIC_GEMFIRE_DATA}, writesToSharedConfiguration=true)
+  @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEODE_REGION, CliStrings.TOPIC_GEODE_DATA }, writesToSharedConfiguration=true)
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   //TODO : Add optionContext for indexName
   public Result createDefinedIndexes(
@@ -634,7 +634,7 @@ public class IndexCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.CLEAR_DEFINED_INDEXES, help = CliStrings.CLEAR_DEFINED__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEMFIRE_REGION, CliStrings.TOPIC_GEMFIRE_DATA}, writesToSharedConfiguration=true)
+  @CliMetaData(shellOnly = false, relatedTopic={CliStrings.TOPIC_GEODE_REGION, CliStrings.TOPIC_GEODE_DATA }, writesToSharedConfiguration=true)
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   //TODO : Add optionContext for indexName
   public Result clearDefinedIndexes() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommands.java
index c67a4bc..6979ed8 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/LauncherLifecycleCommands.java
@@ -146,7 +146,7 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.START_LOCATOR, help = CliStrings.START_LOCATOR__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEMFIRE_LOCATOR, CliStrings.TOPIC_GEMFIRE_LIFECYCLE })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEODE_LOCATOR, CliStrings.TOPIC_GEODE_LIFECYCLE })
   public Result startLocator(
       @CliOption(key = CliStrings.START_LOCATOR__MEMBER_NAME,
           mandatory = true,
@@ -266,14 +266,14 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
       gemfirePropertiesPathname = CliUtil.resolvePathname(gemfirePropertiesPathname);
 
       if (!StringUtils.isBlank(gemfirePropertiesPathname) && !IOUtils.isExistingPathname(gemfirePropertiesPathname)) {
-        return ResultBuilder.createUserErrorResult(CliStrings.format(CliStrings.GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE,
+        return ResultBuilder.createUserErrorResult(CliStrings.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE,
             StringUtils.EMPTY_STRING, gemfirePropertiesPathname));
       }
 
       gemfireSecurityPropertiesPathname = CliUtil.resolvePathname(gemfireSecurityPropertiesPathname);
 
       if (!StringUtils.isBlank(gemfireSecurityPropertiesPathname) && !IOUtils.isExistingPathname(gemfireSecurityPropertiesPathname)) {
-        return ResultBuilder.createUserErrorResult(CliStrings.format(CliStrings.GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE,
+        return ResultBuilder.createUserErrorResult(CliStrings.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE,
             "Security ", gemfireSecurityPropertiesPathname));
       }
 
@@ -714,7 +714,7 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.STATUS_LOCATOR, help = CliStrings.STATUS_LOCATOR__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEMFIRE_LOCATOR, CliStrings.TOPIC_GEMFIRE_LIFECYCLE })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEODE_LOCATOR, CliStrings.TOPIC_GEODE_LIFECYCLE })
   public Result statusLocator(@CliOption(key = CliStrings.STATUS_LOCATOR__MEMBER,
       optionContext = ConverterHint.LOCATOR_MEMBER_IDNAME,
       unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
@@ -782,7 +782,7 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.STOP_LOCATOR, help = CliStrings.STOP_LOCATOR__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEMFIRE_LOCATOR, CliStrings.TOPIC_GEMFIRE_LIFECYCLE })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEODE_LOCATOR, CliStrings.TOPIC_GEODE_LIFECYCLE })
   @ResourceOperation(resource = OperationContext.Resource.CLUSTER, operation = OperationContext.OperationCode.MANAGE)
   public Result stopLocator(@CliOption(key = CliStrings.STOP_LOCATOR__MEMBER,
       optionContext = ConverterHint.LOCATOR_MEMBER_IDNAME,
@@ -1248,7 +1248,7 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.START_SERVER, help = CliStrings.START_SERVER__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEMFIRE_SERVER, CliStrings.TOPIC_GEMFIRE_LIFECYCLE })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEODE_SERVER, CliStrings.TOPIC_GEODE_LIFECYCLE })
   public Result startServer(@CliOption(key = CliStrings.START_SERVER__ASSIGN_BUCKETS,
       unspecifiedDefaultValue = "false",
       specifiedDefaultValue = "true",
@@ -1478,14 +1478,14 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
       gemfirePropertiesPathname = CliUtil.resolvePathname(gemfirePropertiesPathname);
 
       if (!StringUtils.isBlank(gemfirePropertiesPathname) && !IOUtils.isExistingPathname(gemfirePropertiesPathname)) {
-        return ResultBuilder.createUserErrorResult(CliStrings.format(CliStrings.GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE,
+        return ResultBuilder.createUserErrorResult(CliStrings.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE,
             StringUtils.EMPTY_STRING, gemfirePropertiesPathname));
       }
 
       gemfireSecurityPropertiesPathname = CliUtil.resolvePathname(gemfireSecurityPropertiesPathname);
 
       if (!StringUtils.isBlank(gemfireSecurityPropertiesPathname) && !IOUtils.isExistingPathname(gemfireSecurityPropertiesPathname)) {
-        return ResultBuilder.createUserErrorResult(CliStrings.format(CliStrings.GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE,
+        return ResultBuilder.createUserErrorResult(CliStrings.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE,
             "Security ", gemfireSecurityPropertiesPathname));
       }
 
@@ -1804,7 +1804,7 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.STATUS_SERVER, help = CliStrings.STATUS_SERVER__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEMFIRE_SERVER, CliStrings.TOPIC_GEMFIRE_LIFECYCLE })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEODE_SERVER, CliStrings.TOPIC_GEODE_LIFECYCLE })
   public Result statusServer(@CliOption(key = CliStrings.STATUS_SERVER__MEMBER,
       optionContext = ConverterHint.MEMBERIDNAME,
       unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
@@ -1866,7 +1866,7 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.STOP_SERVER, help = CliStrings.STOP_SERVER__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEMFIRE_SERVER, CliStrings.TOPIC_GEMFIRE_LIFECYCLE })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEODE_SERVER, CliStrings.TOPIC_GEODE_LIFECYCLE })
   @ResourceOperation(resource = OperationContext.Resource.CLUSTER, operation = OperationContext.OperationCode.MANAGE)
   public Result stopServer(@CliOption(key = CliStrings.STOP_SERVER__MEMBER,
       optionContext = ConverterHint.MEMBERIDNAME,
@@ -1954,7 +1954,7 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
   }
 
   //@CliCommand(value=CliStrings.START_MANAGER, help=CliStrings.START_MANAGER__HELP)
-  //@CliMetaData(shellOnly=true, relatedTopic = {CliStrings.TOPIC_GEMFIRE_MANAGER, CliStrings.TOPIC_GEMFIRE_JMX, CliStrings.TOPIC_GEMFIRE_LIFECYCLE})
+  //@CliMetaData(shellOnly=true, relatedTopic = {CliStrings.TOPIC_GEODE_MANAGER, CliStrings.TOPIC_GEODE_JMX, CliStrings.TOPIC_GEODE_LIFECYCLE})
   public Result startManager(@CliOption(key = CliStrings.START_MANAGER__MEMBERNAME,
       unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
       help = CliStrings.START_MANAGER__MEMBERNAME__HELP)
@@ -1987,15 +1987,15 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
           help = CliStrings.START_MANAGER__J__HELP)
           Map<String, String> systepProps,
-      @CliOption(key = CliStrings.START_MANAGER__GEMFIREPROPS,
+      @CliOption(key = CliStrings.START_MANAGER__GEODEPROPS,
           unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
-          help = CliStrings.START_MANAGER__GEMFIREPROPS__HELP)
+          help = CliStrings.START_MANAGER__GEODEPROPS__HELP)
           Map<String, String> gemfireProps) {
     return ResultBuilder.createInfoResult("Not-implemented");
   }
 
   @CliCommand(value = CliStrings.START_JCONSOLE, help = CliStrings.START_JCONSOLE__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEMFIRE_MANAGER, CliStrings.TOPIC_GEMFIRE_JMX, CliStrings.TOPIC_GEMFIRE_M_AND_M })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEODE_MANAGER, CliStrings.TOPIC_GEODE_JMX, CliStrings.TOPIC_GEODE_M_AND_M })
   public Result startJConsole(@CliOption(key = CliStrings.START_JCONSOLE__INTERVAL,
       unspecifiedDefaultValue = "4",
       help = CliStrings.START_JCONSOLE__INTERVAL__HELP)
@@ -2174,7 +2174,7 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.START_JVISUALVM, help = CliStrings.START_JVISUALVM__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEMFIRE_MANAGER, CliStrings.TOPIC_GEMFIRE_JMX, CliStrings.TOPIC_GEMFIRE_M_AND_M })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEODE_MANAGER, CliStrings.TOPIC_GEODE_JMX, CliStrings.TOPIC_GEODE_M_AND_M })
   public Result startJVisualVM(@CliOption(key = CliStrings.START_JCONSOLE__J,
       optionContext = ConverterHint.STRING_LIST,
       unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE,
@@ -2254,7 +2254,7 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.START_PULSE, help = CliStrings.START_PULSE__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEMFIRE_MANAGER, CliStrings.TOPIC_GEMFIRE_JMX, CliStrings.TOPIC_GEMFIRE_M_AND_M })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEODE_MANAGER, CliStrings.TOPIC_GEODE_JMX, CliStrings.TOPIC_GEODE_M_AND_M })
   // TODO change url parameter type to URL when I figure out the Converter logic in Gfsh
   public Result startPulse(@CliOption(key = CliStrings.START_PULSE__URL,
       unspecifiedDefaultValue = "http://localhost:7070/pulse",
@@ -2336,13 +2336,13 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
   }
 
   @CliCommand(value = CliStrings.START_VSD, help = CliStrings.START_VSD__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEMFIRE_M_AND_M, CliStrings.TOPIC_GEMFIRE_STATISTICS })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEODE_M_AND_M, CliStrings.TOPIC_GEODE_STATISTICS })
   public Result startVsd(@CliOption(key = CliStrings.START_VSD__FILE, help = CliStrings.START_VSD__FILE__HELP)
   final String[] statisticsArchiveFilePathnames) {
     try {
       String gemfireHome = System.getenv("GEMFIRE");
 
-      assertState(!StringUtils.isBlank(gemfireHome), CliStrings.GEMFIRE_HOME_NOT_FOUND_ERROR_MESSAGE);
+      assertState(!StringUtils.isBlank(gemfireHome), CliStrings.GEODE_HOME_NOT_FOUND_ERROR_MESSAGE);
 
       assertState(IOUtils.isExistingPathname(getPathToVsd()), String.format(CliStrings.START_VSD__NOT_FOUND_ERROR_MESSAGE,
           gemfireHome));
@@ -2447,12 +2447,12 @@ public class LauncherLifecycleCommands extends AbstractCommandsSupport {
 
   // NOTE as of 8.0, this command is no more!
   //@CliCommand(value=CliStrings.START_DATABROWSER, help=CliStrings.START_DATABROWSER__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEMFIRE_M_AND_M })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEODE_M_AND_M })
   public Result startDataBrowser() {
     try {
       String gemfireHome = System.getenv("GEMFIRE");
 
-      assertState(!StringUtils.isBlank(gemfireHome), CliStrings.GEMFIRE_HOME_NOT_FOUND_ERROR_MESSAGE);
+      assertState(!StringUtils.isBlank(gemfireHome), CliStrings.GEODE_HOME_NOT_FOUND_ERROR_MESSAGE);
 
       if (isConnectedAndReady() && (getGfsh().getOperationInvoker() instanceof JmxOperationInvoker)) {
         String dataBrowserPath = getPathToDataBrowser();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommands.java
index 7db1deb..08e36a4 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/MemberCommands.java
@@ -63,7 +63,7 @@ public class MemberCommands implements CommandMarker {
   private static final GetMemberInformationFunction getMemberInformation = new GetMemberInformationFunction();
 
   @CliCommand(value = { CliStrings.LIST_MEMBER }, help = CliStrings.LIST_MEMBER__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = CliStrings.TOPIC_GEMFIRE_SERVER)
+  @CliMetaData(shellOnly = false, relatedTopic = CliStrings.TOPIC_GEODE_SERVER)
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result listMember(
 		  @CliOption(key = { CliStrings.LIST_MEMBER__GROUP },
@@ -108,7 +108,7 @@ public class MemberCommands implements CommandMarker {
   }
 
   @CliCommand(value = { CliStrings.DESCRIBE_MEMBER }, help = CliStrings.DESCRIBE_MEMBER__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = CliStrings.TOPIC_GEMFIRE_SERVER)
+  @CliMetaData(shellOnly = false, relatedTopic = CliStrings.TOPIC_GEODE_SERVER)
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result describeMember(
   	@CliOption(key = CliStrings.DESCRIBE_MEMBER__IDENTIFIER,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommands.java
index 020b134..2f48e22 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/MiscellaneousCommands.java
@@ -183,7 +183,7 @@ public class MiscellaneousCommands implements CommandMarker {
 
 
   @CliCommand(value = CliStrings.SHUTDOWN, help = CliStrings.SHUTDOWN__HELP)
-  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEMFIRE_LIFECYCLE },
+  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEODE_LIFECYCLE },
       interceptor = "com.gemstone.gemfire.management.internal.cli.commands.MiscellaneousCommands$Interceptor")
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.MANAGE)
   public Result shutdown(
@@ -325,7 +325,7 @@ public class MiscellaneousCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.GC, help = CliStrings.GC__HELP)
-  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEMFIRE_DEBUG_UTIL })
+  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEODE_DEBUG_UTIL })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.MANAGE)
   public Result gc(
       @CliOption(key = CliStrings.GC__GROUP, unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, help = CliStrings.GC__GROUP__HELP)
@@ -416,7 +416,7 @@ public class MiscellaneousCommands implements CommandMarker {
 
 
   @CliCommand(value = CliStrings.NETSTAT, help = CliStrings.NETSTAT__HELP)
-  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEMFIRE_DEBUG_UTIL })
+  @CliMetaData(relatedTopic = { CliStrings.TOPIC_GEODE_DEBUG_UTIL })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   //TODO : Verify the auto-completion for multiple values.
   public Result netstat(
@@ -570,7 +570,7 @@ public class MiscellaneousCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.SHOW_DEADLOCK, help = CliStrings.SHOW_DEADLOCK__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEMFIRE_DEBUG_UTIL })
+  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEODE_DEBUG_UTIL })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result showDeadlock(
       @CliOption(key = CliStrings.SHOW_DEADLOCK__DEPENDENCIES__FILE,
@@ -620,7 +620,7 @@ public class MiscellaneousCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.SHOW_LOG, help = CliStrings.SHOW_LOG_HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEMFIRE_DEBUG_UTIL })
+  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEODE_DEBUG_UTIL })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result showLog(
       @CliOption(key = CliStrings.SHOW_LOG_MEMBER, optionContext = ConverterHint.ALL_MEMBER_IDNAME, unspecifiedDefaultValue = CliMetaData.ANNOTATION_NULL_VALUE, help = CliStrings.SHOW_LOG_MEMBER_HELP, mandatory = true) String memberNameOrId,
@@ -768,7 +768,7 @@ public class MiscellaneousCommands implements CommandMarker {
     return result;
   }
   @CliCommand(value = CliStrings.EXPORT_LOGS, help = CliStrings.EXPORT_LOGS__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEMFIRE_SERVER, CliStrings.TOPIC_GEMFIRE_DEBUG_UTIL })
+  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEODE_SERVER, CliStrings.TOPIC_GEODE_DEBUG_UTIL })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result exportLogs(
       @CliOption(key = CliStrings.EXPORT_LOGS__DIR,
@@ -990,7 +990,7 @@ public class MiscellaneousCommands implements CommandMarker {
    * @return Stack Trace
    */
   @CliCommand(value = CliStrings.EXPORT_STACKTRACE, help = CliStrings.EXPORT_STACKTRACE__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEMFIRE_DEBUG_UTIL })
+  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEODE_DEBUG_UTIL })
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result exportStackTrace(
       @CliOption(key = CliStrings.EXPORT_STACKTRACE__MEMBER,
@@ -1091,7 +1091,7 @@ public class MiscellaneousCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.SHOW_METRICS, help = CliStrings.SHOW_METRICS__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEMFIRE_STATISTICS })
+  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEODE_STATISTICS })
   @ResourceOperation(resource = Resource.CLUSTER, operation= OperationCode.READ)
   public Result showMetrics(
       @CliOption(key = { CliStrings.SHOW_METRICS__MEMBER }, optionContext = ConverterHint.ALL_MEMBER_IDNAME, help = CliStrings.SHOW_METRICS__MEMBER__HELP) String memberNameOrId,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/PDXCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/PDXCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/PDXCommands.java
index 52af8f1..b9ffd02 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/PDXCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/PDXCommands.java
@@ -52,7 +52,7 @@ public class PDXCommands extends AbstractCommandsSupport{
 
 
   @CliCommand (value = CliStrings.CONFIGURE_PDX, help = CliStrings.CONFIGURE_PDX__HELP)
-  @CliMetaData (relatedTopic = CliStrings.TOPIC_GEMFIRE_REGION, writesToSharedConfiguration = true)
+  @CliMetaData (relatedTopic = CliStrings.TOPIC_GEODE_REGION, writesToSharedConfiguration = true)
   @ResourceOperation( resource=Resource.DATA, operation = OperationCode.MANAGE)
   public Result configurePDX(
       @CliOption (key = CliStrings.CONFIGURE_PDX__READ__SERIALIZED,
@@ -174,7 +174,7 @@ public class PDXCommands extends AbstractCommandsSupport{
   }
 
   @CliCommand (value = CliStrings.PDX_RENAME, help = CliStrings.PDX_RENAME__HELP)
-  @CliMetaData(shellOnly=true, relatedTopic={CliStrings.TOPIC_GEMFIRE_DISKSTORE})
+  @CliMetaData(shellOnly=true, relatedTopic={CliStrings.TOPIC_GEODE_DISKSTORE })
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public Result pdxRename(
       @CliOption (key = CliStrings.PDX_RENAME_OLD,
@@ -230,7 +230,7 @@ public class PDXCommands extends AbstractCommandsSupport{
   
   // The pdx delete-field command has been disabled until it has native client support.
 //  @CliCommand (value = CliStrings.PDX_DELETE_FIELD, help = CliStrings.PDX_DELETE_FIELD__HELP)
-//  @CliMetaData(shellOnly=true, relatedTopic={CliStrings.TOPIC_GEMFIRE_DISKSTORE})
+//  @CliMetaData(shellOnly=true, relatedTopic={CliStrings.TOPIC_GEODE_DISKSTORE})
 //  public Result pdxDeleteField( 
 //      @CliOption (key = CliStrings.PDX_CLASS,
 //      mandatory=true,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/RegionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/RegionCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/RegionCommands.java
index 93d55f4..63f0ea2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/RegionCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/RegionCommands.java
@@ -76,7 +76,7 @@ public class RegionCommands implements CommandMarker {
   private static final GetRegionDescriptionFunction getRegionDescription = new GetRegionDescriptionFunction();
 
   @CliCommand(value = { CliStrings.LIST_REGION }, help = CliStrings.LIST_REGION__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = CliStrings.TOPIC_GEMFIRE_REGION)
+  @CliMetaData(shellOnly = false, relatedTopic = CliStrings.TOPIC_GEODE_REGION)
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result listRegion(
       @CliOption(key = { CliStrings.LIST_REGION__GROUP },
@@ -154,7 +154,7 @@ public class RegionCommands implements CommandMarker {
   }
 
   @CliCommand(value = { CliStrings.DESCRIBE_REGION }, help = CliStrings.DESCRIBE_REGION__HELP)
-  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEMFIRE_REGION, CliStrings.TOPIC_GEMFIRE_CONFIG } )
+  @CliMetaData(shellOnly = false, relatedTopic = { CliStrings.TOPIC_GEODE_REGION, CliStrings.TOPIC_GEODE_CONFIG } )
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result describeRegion(
       @CliOption(key = CliStrings.DESCRIBE_REGION__NAME,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
index 6b1a330..09a25a6 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/ShellCommands.java
@@ -121,7 +121,7 @@ public class ShellCommands implements CommandMarker {
   }
 
   @CliCommand(value = { CliStrings.CONNECT }, help = CliStrings.CONNECT__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEMFIRE_JMX, CliStrings.TOPIC_GEMFIRE_MANAGER })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEODE_JMX, CliStrings.TOPIC_GEODE_MANAGER })
   public Result connect(
     @CliOption(key = { CliStrings.CONNECT__LOCATOR },
       unspecifiedDefaultValue = ConnectionEndpointConverter.DEFAULT_LOCATOR_ENDPOINTS,
@@ -478,7 +478,7 @@ public class ShellCommands implements CommandMarker {
       if (!StringUtils.isBlank(gfSecurityPropertiesPathToUse)) {
         // User specified gfSecurity properties doesn't exist
         if (!IOUtils.isExistingPathname(gfSecurityPropertiesPathToUse)) {
-          gfshInstance.printAsSevere(CliStrings.format(CliStrings.GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, "Security ", gfSecurityPropertiesPathToUse));
+          gfshInstance.printAsSevere(CliStrings.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, "Security ", gfSecurityPropertiesPathToUse));
         }
         else {
           gfSecurityPropertiesUrl = new File(gfSecurityPropertiesPathToUse).toURI().toURL();
@@ -701,7 +701,7 @@ public class ShellCommands implements CommandMarker {
 
 
   @CliCommand(value = { CliStrings.DISCONNECT }, help = CliStrings.DISCONNECT__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEMFIRE_JMX, CliStrings.TOPIC_GEMFIRE_MANAGER })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEODE_JMX, CliStrings.TOPIC_GEODE_MANAGER })
   public Result disconnect() {
     Result result = null;
 
@@ -736,7 +736,7 @@ public class ShellCommands implements CommandMarker {
 
 
   @CliCommand(value = { CliStrings.DESCRIBE_CONNECTION }, help = CliStrings.DESCRIBE_CONNECTION__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEMFIRE_JMX })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEODE_JMX })
   public Result describeConnection() {
     Result result = null;
     try {
@@ -831,7 +831,7 @@ public class ShellCommands implements CommandMarker {
 
   //Enable when "use region" command is required. See #46110
   //  @CliCommand(value = { CliStrings.USE_REGION }, help = CliStrings.USE_REGION__HELP)
-  //  @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEMFIRE_REGION})
+  //  @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEODE_REGION})
   //  public Result useRegion(
   //      @CliArgument(name = CliStrings.USE_REGION__REGION,
   //                   unspecifiedDefaultValue = "/",
@@ -845,7 +845,7 @@ public class ShellCommands implements CommandMarker {
   //  }
 
   @CliCommand(value = { CliStrings.DEBUG }, help = CliStrings.DEBUG__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEMFIRE_DEBUG_UTIL })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GFSH, CliStrings.TOPIC_GEODE_DEBUG_UTIL })
   public Result debug(
     @CliOption(key = CliStrings.DEBUG__STATE,
       unspecifiedDefaultValue = "OFF",
@@ -1026,7 +1026,7 @@ public class ShellCommands implements CommandMarker {
 
 
   @CliCommand(value = CliStrings.ENCRYPT, help = CliStrings.ENCRYPT__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEMFIRE_DEBUG_UTIL })
+  @CliMetaData(shellOnly = true, relatedTopic = { CliStrings.TOPIC_GEODE_DEBUG_UTIL })
   public Result encryptPassword(
     @CliOption(key = CliStrings.ENCRYPT_STRING,
       help = CliStrings.ENCRYPT_STRING__HELP,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/StatusCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/StatusCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/StatusCommands.java
index bf0b044..ba4b0ac 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/StatusCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/StatusCommands.java
@@ -47,7 +47,7 @@ public class StatusCommands extends AbstractCommandsSupport implements CommandMa
 
   @SuppressWarnings("unchecked")
   @CliCommand (value = CliStrings.STATUS_SHARED_CONFIG, help = CliStrings.STATUS_SHARED_CONFIG_HELP)
-  @CliMetaData (relatedTopic = CliStrings.TOPIC_GEMFIRE_LOCATOR)
+  @CliMetaData (relatedTopic = CliStrings.TOPIC_GEODE_LOCATOR)
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result statusSharedConfiguration() {
     final GemFireCacheImpl cache = GemFireCacheImpl.getInstance();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/WanCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/WanCommands.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/WanCommands.java
index 2e9fed1..47b547d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/WanCommands.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/WanCommands.java
@@ -75,7 +75,7 @@ public class WanCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.CREATE_GATEWAYSENDER, help = CliStrings.CREATE_GATEWAYSENDER__HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEMFIRE_WAN, writesToSharedConfiguration=true)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN, writesToSharedConfiguration=true)
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public Result createGatewaySender(
       @CliOption(key = CliStrings.CREATE_GATEWAYSENDER__GROUP,
@@ -192,7 +192,7 @@ public class WanCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.START_GATEWAYSENDER, help = CliStrings.START_GATEWAYSENDER__HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEMFIRE_WAN)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN)
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public Result startGatewaySender(
       @CliOption(key = CliStrings.START_GATEWAYSENDER__ID,
@@ -335,7 +335,7 @@ public class WanCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.PAUSE_GATEWAYSENDER, help = CliStrings.PAUSE_GATEWAYSENDER__HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEMFIRE_WAN)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN)
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public Result pauseGatewaySender(
       @CliOption(key = CliStrings.PAUSE_GATEWAYSENDER__ID,
@@ -428,7 +428,7 @@ public class WanCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.RESUME_GATEWAYSENDER, help = CliStrings.RESUME_GATEWAYSENDER__HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEMFIRE_WAN)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN)
   @ResourceOperation(resource=Resource.DATA, operation = OperationCode.MANAGE)
   public Result resumeGatewaySender(
       @CliOption(key = CliStrings.RESUME_GATEWAYSENDER__ID,
@@ -568,7 +568,7 @@ public class WanCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.STOP_GATEWAYSENDER, help = CliStrings.STOP_GATEWAYSENDER__HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEMFIRE_WAN)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN)
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public Result stopGatewaySender(
       @CliOption(key = CliStrings.STOP_GATEWAYSENDER__ID,
@@ -644,7 +644,7 @@ public class WanCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.CREATE_GATEWAYRECEIVER, help = CliStrings.CREATE_GATEWAYRECEIVER__HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEMFIRE_WAN)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN)
   @ResourceOperation( resource=Resource.DATA, operation = OperationCode.MANAGE)
   public Result createGatewayReceiver(
       @CliOption(key = CliStrings.CREATE_GATEWAYRECEIVER__GROUP,
@@ -724,7 +724,7 @@ public class WanCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.LOAD_BALANCE_GATEWAYSENDER, help = CliStrings.LOAD_BALANCE_GATEWAYSENDER__HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEMFIRE_WAN)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN)
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public Result loadBalanceGatewaySender(
       @CliOption(key = CliStrings.LOAD_BALANCE_GATEWAYSENDER__ID,
@@ -790,7 +790,7 @@ public class WanCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.START_GATEWAYRECEIVER, help = CliStrings.START_GATEWAYRECEIVER__HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEMFIRE_WAN)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN)
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public Result startGatewayReceiver(
       @CliOption(key = CliStrings.START_GATEWAYRECEIVER__GROUP,
@@ -852,7 +852,7 @@ public class WanCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.STOP_GATEWAYRECEIVER, help = CliStrings.STOP_GATEWAYRECEIVER__HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEMFIRE_WAN)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN)
   @ResourceOperation(resource = Resource.DATA, operation = OperationCode.MANAGE)
   public Result stopGatewayReceiver(
 
@@ -926,7 +926,7 @@ public class WanCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.LIST_GATEWAY, help = CliStrings.LIST_GATEWAY__HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEMFIRE_WAN)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN)
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result listGateway(
       @CliOption(key = CliStrings.LIST_GATEWAY__MEMBER,
@@ -1017,7 +1017,7 @@ public class WanCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.STATUS_GATEWAYSENDER, help = CliStrings.STATUS_GATEWAYSENDER__HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEMFIRE_WAN)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN)
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result statusGatewaySender(
       @CliOption(key = CliStrings.STATUS_GATEWAYSENDER__ID,
@@ -1084,7 +1084,7 @@ public class WanCommands implements CommandMarker {
   }
 
   @CliCommand(value = CliStrings.STATUS_GATEWAYRECEIVER, help = CliStrings.STATUS_GATEWAYRECEIVER__HELP)
-  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEMFIRE_WAN)
+  @CliMetaData(relatedTopic = CliStrings.TOPIC_GEODE_WAN)
   @ResourceOperation(resource = Resource.CLUSTER, operation = OperationCode.READ)
   public Result statusGatewayReceiver(
       @CliOption(key = CliStrings.STATUS_GATEWAYRECEIVER__GROUP,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/help/CliTopic.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/help/CliTopic.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/help/CliTopic.java
index 4b93a37..2849a3f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/help/CliTopic.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/help/CliTopic.java
@@ -34,22 +34,22 @@ public class CliTopic implements Comparable<CliTopic> {
   private static final Map<String, String> nameDescriptionMap = new HashMap<String, String>();
 
   static {
-    nameDescriptionMap.put(CliStrings.DEFAULT_TOPIC_GEMFIRE,    CliStrings.DEFAULT_TOPIC_GEMFIRE__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_REGION,     CliStrings.TOPIC_GEMFIRE_REGION__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_WAN,        CliStrings.TOPIC_GEMFIRE_WAN__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_JMX,        CliStrings.TOPIC_GEMFIRE_JMX__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_DISKSTORE,  CliStrings.TOPIC_GEMFIRE_DISKSTORE__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_LOCATOR,    CliStrings.TOPIC_GEMFIRE_LOCATOR__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_SERVER,     CliStrings.TOPIC_GEMFIRE_SERVER__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_MANAGER,    CliStrings.TOPIC_GEMFIRE_MANAGER__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_STATISTICS, CliStrings.TOPIC_GEMFIRE_STATISTICS__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_LIFECYCLE,  CliStrings.TOPIC_GEMFIRE_LIFECYCLE__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_M_AND_M,    CliStrings.TOPIC_GEMFIRE_M_AND_M__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_DATA,       CliStrings.TOPIC_GEMFIRE_DATA__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_CONFIG,     CliStrings.TOPIC_GEMFIRE_CONFIG__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_FUNCTION,   CliStrings.TOPIC_GEMFIRE_FUNCTION__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_HELP,       CliStrings.TOPIC_GEMFIRE_HELP__DESC);
-    nameDescriptionMap.put(CliStrings.TOPIC_GEMFIRE_DEBUG_UTIL, CliStrings.TOPIC_GEMFIRE_DEBUG_UTIL__DESC);
+    nameDescriptionMap.put(CliStrings.DEFAULT_TOPIC_GEODE,    CliStrings.DEFAULT_TOPIC_GEODE__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_REGION,     CliStrings.TOPIC_GEODE_REGION__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_WAN,        CliStrings.TOPIC_GEODE_WAN__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_JMX,        CliStrings.TOPIC_GEODE_JMX__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_DISKSTORE,  CliStrings.TOPIC_GEODE_DISKSTORE__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_LOCATOR,    CliStrings.TOPIC_GEODE_LOCATOR__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_SERVER,     CliStrings.TOPIC_GEODE_SERVER__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_MANAGER,    CliStrings.TOPIC_GEODE_MANAGER__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_STATISTICS, CliStrings.TOPIC_GEODE_STATISTICS__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_LIFECYCLE,  CliStrings.TOPIC_GEODE_LIFECYCLE__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_M_AND_M,    CliStrings.TOPIC_GEODE_M_AND_M__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_DATA,       CliStrings.TOPIC_GEODE_DATA__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_CONFIG,     CliStrings.TOPIC_GEODE_CONFIG__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_FUNCTION,   CliStrings.TOPIC_GEODE_FUNCTION__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_HELP,       CliStrings.TOPIC_GEODE_HELP__DESC);
+    nameDescriptionMap.put(CliStrings.TOPIC_GEODE_DEBUG_UTIL, CliStrings.TOPIC_GEODE_DEBUG_UTIL__DESC);
     nameDescriptionMap.put(CliStrings.TOPIC_GFSH,               CliStrings.TOPIC_GFSH__DESC);
   }
 


[05/16] incubator-geode git commit: GEODE-11: Removed ParseException from LuceneQueryFactory.create, added LuceneQueryException

Posted by kl...@apache.org.
GEODE-11: Removed ParseException from LuceneQueryFactory.create, added LuceneQueryException

Adding a new exception to wrap lucene exceptions called LuceneQueryException. Removed lucene exceptions from the public API.


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

Branch: refs/heads/feature/GEODE-1565
Commit: 786c862e1c158859b90d52ee0d053507b65e8a01
Parents: 28d2ce0
Author: Aparna Dharmakkan <ad...@pivotal.io>
Authored: Thu Jun 16 11:20:13 2016 -0700
Committer: nabarun <nn...@pivotal.io>
Committed: Thu Jun 16 16:14:35 2016 -0700

----------------------------------------------------------------------
 .../cache/lucene/LuceneQueryException.java      | 36 ++++++++++++++++++++
 .../cache/lucene/LuceneQueryFactory.java        |  4 +--
 .../cache/lucene/LuceneQueryProvider.java       |  5 +--
 .../lucene/internal/StringQueryProvider.java    |  8 +++--
 .../internal/distributed/LuceneFunction.java    |  3 +-
 .../lucene/LuceneQueriesIntegrationTest.java    |  4 +--
 .../internal/StringQueryProviderJUnitTest.java  | 13 +++----
 .../IndexRepositoryImplPerformanceTest.java     |  5 +--
 8 files changed, 59 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java
new file mode 100644
index 0000000..683b799
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java
@@ -0,0 +1,36 @@
+/*
+ * 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.lucene;
+
+import com.gemstone.gemfire.GemFireCheckedException;
+
+/**
+ * Thrown when a lucene query fails.
+ */
+public class LuceneQueryException extends GemFireCheckedException {
+
+  public LuceneQueryException(final String message) {
+    super(message);
+  }
+
+  public LuceneQueryException(final String message, final Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryFactory.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryFactory.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryFactory.java
index a7844a2..8e36bbb 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryFactory.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryFactory.java
@@ -80,10 +80,8 @@ public interface LuceneQueryFactory {
    * @param <K> the key type in the query results
    * @param <V> the value type in the query results
    * @return LuceneQuery object
-   * @throws ParseException
    */
-  public <K, V> LuceneQuery<K, V> create(String indexName, String regionName, String queryString, String defaultField)
-      throws ParseException;
+  public <K, V> LuceneQuery<K, V> create(String indexName, String regionName, String queryString, String defaultField);
 
   /**
    * Creates a wrapper object for Lucene's Query object. This {@link LuceneQuery} builder method could be used in

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java
index 92a3a1c..7f1c269 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java
@@ -39,8 +39,9 @@ public interface LuceneQueryProvider extends Serializable {
   /**
    * @return A Lucene Query object which could be used for executing Lucene Search on indexed data
    * @param index local lucene index the query is being constructed against.
-   * @throws QueryException if the provider fails to construct the query object
+   * @throws LuceneQueryException if the provider fails to construct the query object
    */
-  public Query getQuery(LuceneIndex index) throws QueryException;
+
+  public Query getQuery(LuceneIndex index) throws LuceneQueryException;
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java
index 3f121ec..c5d145e 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java
@@ -29,7 +29,10 @@ import org.apache.lucene.queryparser.flexible.standard.StandardQueryParser;
 import org.apache.lucene.search.Query;
 
 import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.GemFireCheckedException;
+import com.gemstone.gemfire.GemFireException;
 import com.gemstone.gemfire.cache.lucene.LuceneIndex;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
 import com.gemstone.gemfire.cache.query.QueryException;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
@@ -64,10 +67,9 @@ public class StringQueryProvider implements LuceneQueryProvider, DataSerializabl
   }
 
   @Override
-  public synchronized Query getQuery(LuceneIndex index) throws QueryException {
+  public synchronized Query getQuery(LuceneIndex index) throws LuceneQueryException {
     if (luceneQuery == null) {
       String[] fields = index.getFieldNames();
-
       LuceneIndexImpl indexImpl = (LuceneIndexImpl) index;
       StandardQueryParser parser = new StandardQueryParser(indexImpl.getAnalyzer());
       try {
@@ -77,7 +79,7 @@ public class StringQueryProvider implements LuceneQueryProvider, DataSerializabl
         }
       } catch (QueryNodeException e) {
         logger.debug("Query node exception:" + query, e);
-        throw new QueryException(e);
+        throw new LuceneQueryException("Malformed lucene query: " + query, e);
       }
     }
     return luceneQuery;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunction.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunction.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunction.java
index 9567305..3c6c0d2 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunction.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunction.java
@@ -32,6 +32,7 @@ import com.gemstone.gemfire.cache.execute.FunctionContext;
 import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.execute.RegionFunctionContext;
 import com.gemstone.gemfire.cache.execute.ResultSender;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
@@ -80,7 +81,7 @@ public class LuceneFunction extends FunctionAdapter implements InternalEntity {
     Query query = null;
     try {
       query = queryProvider.getQuery(index);
-    } catch (QueryException e) {
+    } catch (LuceneQueryException e) {
       logger.warn("", e);
       throw new FunctionException(e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
index 674bc9c..bfb8c88 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
@@ -208,7 +208,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
     //Create a query that throws an exception
     final LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
       (index) -> {
-        throw new QueryException("Bad query");
+        throw new LuceneQueryException("Bad query");
       });
 
 
@@ -217,7 +217,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
     try {
       query.search();
     } catch(FunctionException e) {
-      assertEquals(QueryException.class, e.getCause().getClass());
+      assertEquals(LuceneQueryException.class, e.getCause().getClass());
       throw e;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
index a08875a..332ce35 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
@@ -32,7 +32,8 @@ import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
 import com.gemstone.gemfire.CopyHelper;
-import com.gemstone.gemfire.cache.query.QueryException;
+
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -52,7 +53,7 @@ public class StringQueryProviderJUnitTest {
   }
 
   @Test
-  public void testQueryConstruction() throws QueryException {
+  public void testQueryConstruction() throws LuceneQueryException {
     StringQueryProvider provider = new StringQueryProvider("foo:bar", DEFAULT_FIELD);
     Query query = provider.getQuery(mockIndex);
     Assert.assertNotNull(query);
@@ -61,15 +62,15 @@ public class StringQueryProviderJUnitTest {
 
   @Test
   @Ignore("Custom analyzer not yet supported, this is a duplicate test right now")
-  public void usesCustomAnalyzer() throws QueryException {
+  public void usesCustomAnalyzer() throws LuceneQueryException {
     StringQueryProvider provider = new StringQueryProvider("findThis", DEFAULT_FIELD);
     Query query = provider.getQuery(mockIndex);
     Assert.assertNotNull(query);
     assertEquals("field-1:findthis field-2:findthis", query.toString());
   }
 
-  @Test(expected = QueryException.class)
-  public void errorsOnMalformedQueryString() throws QueryException {
+  @Test(expected = LuceneQueryException.class)
+  public void errorsOnMalformedQueryString() throws LuceneQueryException {
     StringQueryProvider provider = new StringQueryProvider("invalid:lucene:query:string", DEFAULT_FIELD);
     provider.getQuery(mockIndex);
   }
@@ -83,7 +84,7 @@ public class StringQueryProviderJUnitTest {
   }
 
   @Test
-  public void defaultFieldParameterShouldBeUsedByQuery() throws QueryException {
+  public void defaultFieldParameterShouldBeUsedByQuery() throws LuceneQueryException {
     StringQueryProvider provider = new StringQueryProvider("findThis",  "field-2");
     Query query = provider.getQuery(mockIndex);
     Assert.assertNotNull(query);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
index 5e1a104..61f0ec4 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
@@ -32,7 +32,6 @@ import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.FileSystemStats;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
-import com.gemstone.gemfire.cache.query.QueryException;
 import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
@@ -50,6 +49,7 @@ import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -59,6 +59,7 @@ import java.util.concurrent.TimeUnit;
 
 import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
 
+
 /**
  * Microbenchmark of the IndexRepository to compare an
  * IndexRepository built on top of cache with a 
@@ -190,7 +191,7 @@ public class IndexRepositoryImplPerformanceTest {
         LuceneQuery<Object, Object> luceneQuery = service.createLuceneQueryFactory().create("index", "/region", new LuceneQueryProvider() {
           
           @Override
-          public Query getQuery(LuceneIndex index) throws QueryException {
+          public Query getQuery(LuceneIndex index) throws LuceneQueryException {
             return query;
           }
         });


[11/16] incubator-geode git commit: GEODE-1551: apply FlakyTest category to testSimulateForEntireDS

Posted by kl...@apache.org.
GEODE-1551: apply FlakyTest category to testSimulateForEntireDS


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

Branch: refs/heads/feature/GEODE-1565
Commit: fc766082165785836b4c2fa4bfdb108db9708237
Parents: 6ffef33
Author: Kirk Lund <kl...@apache.org>
Authored: Fri Jun 17 11:47:54 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Jun 17 13:19:47 2016 -0700

----------------------------------------------------------------------
 .../internal/cli/commands/GemfireDataCommandsDUnitTest.java         | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fc766082/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
index 4914681..553b7a4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
@@ -1810,6 +1810,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
+  @Category(FlakyTest.class) // GEODE-1551: org.eclipse.jetty.io.EofException
   @Test
   public void testSimulateForEntireDS() {
     setupTestRebalanceForEntireDS();


[16/16] incubator-geode git commit: GEODE-1565: exclude GemfireDataCommandsDUnitTest over HTTP

Posted by kl...@apache.org.
GEODE-1565: exclude GemfireDataCommandsDUnitTest over HTTP

Testing GemfireDataCommandsDUnitTest over HTTP is enabled on feature-GEODE-1565 and will be merged back to develop as soon as it's passing again.


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

Branch: refs/heads/feature/GEODE-1565
Commit: db739b221d2603eb7984ec24a79338c8efdf53e2
Parents: 832ddd1
Author: Kirk Lund <kl...@apache.org>
Authored: Sat Jun 18 13:44:24 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Sat Jun 18 13:44:24 2016 -0700

----------------------------------------------------------------------
 .../internal/cli/commands/CommandOverHttpDUnitTest.java           | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/db739b22/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java
index 6cc475e..cad3f31 100644
--- a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java
+++ b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java
@@ -14,7 +14,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package com.gemstone.gemfire.management.internal.cli.commands;
 
 import org.junit.experimental.categories.Category;
@@ -43,7 +42,7 @@ import org.junit.runners.Suite;
   DeployCommandsDUnitTest.class,
   DiskStoreCommandsDUnitTest.class,
   FunctionCommandsDUnitTest.class,
-  GemfireDataCommandsDUnitTest.class,
+  //GemfireDataCommandsDUnitTest.class, restore when GEODE-1565 is fixed
   GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest.class,
   IndexCommandsDUnitTest.class,
   ListAndDescribeDiskStoreCommandsDUnitTest.class,


[13/16] incubator-geode git commit: GEODE-985: rebrand GFSH help for Apache Geode GEODE-1408: correct alter region help in gfsh

Posted by kl...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
index cff6535..d5ebe4c 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
@@ -28,14 +28,10 @@ import java.text.MessageFormat;
 
 import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
 
-/**-*
- * Contains 'String' constants used as key to the Localized strings to be used
+/**-
+ *  * Contains 'String' constants used as key to the Localized strings to be used
  * in classes under <code>com.gemstone.gemfire.management.internal.cli</code>
  * for Command Line Interface (CLI).
- *
- * @since GemFire 7.0
- */
-/*-
  * NOTES:
  * 1. CONVENTIONS: Defining constants for Command Name, option, argument, help:
  * 1.1 Command Name:
@@ -63,54 +59,56 @@ import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
  *
  * 2. Defining Topic constants:
  * 2.1 The constants' names should begin with "TOPIC_"
- *     E.g. TOPIC_GEMFIRE_REGION
+ *     E.g. TOPIC_GEODE_REGION
  * 2.2 Topic brief description should be defined with suffix "__DESC".
- *     E.g. TOPIC_GEMFIRE_REGION__DESC
+ *     E.g. TOPIC_GEODE_REGION__DESC
  *
  * 3. Order for adding constants: It should be alphabetically sorted at least
  *    on the first name within the current group
+ *
+ * @since GemFire 7.0
  */
 public class CliStrings {
 
   /*-*************************************************************************
    *************                  T O P I C S                  ***************
    ***************************************************************************/
-  public static final String DEFAULT_TOPIC_GEMFIRE = "GemFire";
-  public static final String DEFAULT_TOPIC_GEMFIRE__DESC = "vFabric GemFire is a distributed data management platform providing dynamic scalability, high performance and database-like persistence.";
-  public static final String TOPIC_GEMFIRE_REGION = "Region";
-  public static final String TOPIC_GEMFIRE_REGION__DESC = "A region is the core building block of the vFabric GemFire distributed system. Cached data is organized into regions and all data puts, gets, and querying activities are done against them.";
-  public static final String TOPIC_GEMFIRE_WAN = "WAN";
-  public static final String TOPIC_GEMFIRE_WAN__DESC = "For multiple data centers in remote locations, GemFire provides a WAN gateway to facilitate data sharing. The WAN gateway connects two or more remote sites and then sends asynchronous, batched updates as data is changed.";
-  public static final String TOPIC_GEMFIRE_JMX = "JMX";
-  public static final String TOPIC_GEMFIRE_JMX__DESC = "JMX technology provides the tools for building distributed, Web-based, modular and dynamic solutions for managing and monitoring devices, applications, and service-driven networks.";
-  public static final String TOPIC_GEMFIRE_DISKSTORE = "Disk Store";
-  public static final String TOPIC_GEMFIRE_DISKSTORE__DESC = "Disk stores are used to persist data to disk as a backup to your in-memory copy or as overflow storage when memory use is too high.";
-  public static final String TOPIC_GEMFIRE_LOCATOR = "Locator";
-  public static final String TOPIC_GEMFIRE_LOCATOR__DESC = "JVMs running GemFire discover each other through a TCP service named the locator.";
-  public static final String TOPIC_GEMFIRE_SERVER = "Server";
-  public static final String TOPIC_GEMFIRE_SERVER__DESC = "A server is GemFire cluster member which holds a GemFire cache. Depending on the topology used it can refer to either a system that responds to client requests or a system that is only a peer to other members.";
-  public static final String TOPIC_GEMFIRE_MANAGER = "Manager";
-  public static final String TOPIC_GEMFIRE_MANAGER__DESC = "The Manager is a member which has the additional role of a managing & monitoring the GemFire distributed system.";
-  public static final String TOPIC_GEMFIRE_STATISTICS = "Statistics";
-  public static final String TOPIC_GEMFIRE_STATISTICS__DESC = "Every application and server in a vFabric GemFire distributed system can be configured to perform statistical data collection for analysis.";
-  public static final String TOPIC_GEMFIRE_LIFECYCLE = "Lifecycle";
-  public static final String TOPIC_GEMFIRE_LIFECYCLE__DESC = "Launching, execution and termination of GemFire cluster members such as servers and locators.";
-  public static final String TOPIC_GEMFIRE_M_AND_M = "Management-Monitoring";
-  public static final String TOPIC_GEMFIRE_M_AND_M__DESC = "The management of and monitoring of GemFire systems using GemFire tools, such as vFabric GemFire Pulse or VSD or Data Browser, and JConsole, which is provided with the JDK(TM)";
-  public static final String TOPIC_GEMFIRE_DATA = "Data";
-  public static final String TOPIC_GEMFIRE_DATA__DESC = "User data as stored in regions of the GemFire distributed system.";
-  public static final String TOPIC_GEMFIRE_CONFIG = "Configuration";
-  public static final String TOPIC_GEMFIRE_CONFIG__DESC = "Configuration of vFabric GemFire Cache & Servers/Locators hosting the Cache.";
-  public static final String TOPIC_GEMFIRE_FUNCTION = "Function Execution";
-  public static final String TOPIC_GEMFIRE_FUNCTION__DESC = "The function execution service provides solutions for these application use cases: \n\tAn application that executes a server-side transaction or carries out data updates using the GemFire distributed locking service. \n\tAn application that needs to initialize some of its components once on each server, which might be used later by executed functions. Initialization and startup of a third-party service, such as a messaging service. \n\tAny arbitrary aggregation operation that requires iteration over local data sets that can be done more efficiently through a single call to the cache server. \n\tAny kind of external resource provisioning that can be done by executing a function on a server.";
-  public static final String TOPIC_GEMFIRE_HELP = "Help";
-  public static final String TOPIC_GEMFIRE_HELP__DESC = "Provides usage information for gfsh & its commands.";
-  public static final String TOPIC_GEMFIRE_DEBUG_UTIL = "Debug-Utility";
-  public static final String TOPIC_GEMFIRE_DEBUG_UTIL__DESC = "Debugging aids & utilities to use with vFabric GemFire.";
+  public static final String DEFAULT_TOPIC_GEODE = "Geode";
+  public static final String DEFAULT_TOPIC_GEODE__DESC = "Apache Geode is a distributed data management platform providing dynamic scalability, high performance and database-like persistence.";
+  public static final String TOPIC_GEODE_REGION = "Region";
+  public static final String TOPIC_GEODE_REGION__DESC = "A region is the core building block of the Apache Geode distributed system. Cached data is organized into regions and all data puts, gets, and querying activities are done against them.";
+  public static final String TOPIC_GEODE_WAN = "WAN";
+  public static final String TOPIC_GEODE_WAN__DESC = "For multiple data centers in remote locations, Geode provides a WAN gateway to facilitate data sharing. The WAN gateway connects two or more remote sites and then sends asynchronous, batched updates as data is changed.";
+  public static final String TOPIC_GEODE_JMX = "JMX";
+  public static final String TOPIC_GEODE_JMX__DESC = "JMX technology provides the tools for building distributed, Web-based, modular and dynamic solutions for managing and monitoring devices, applications, and service-driven networks.";
+  public static final String TOPIC_GEODE_DISKSTORE = "Disk Store";
+  public static final String TOPIC_GEODE_DISKSTORE__DESC = "Disk stores are used to persist data to disk as a backup to your in-memory copy or as overflow storage when memory use is too high.";
+  public static final String TOPIC_GEODE_LOCATOR = "Locator";
+  public static final String TOPIC_GEODE_LOCATOR__DESC = "JVMs running Geode discover each other through a TCP service named the locator.";
+  public static final String TOPIC_GEODE_SERVER = "Server";
+  public static final String TOPIC_GEODE_SERVER__DESC = "A server is Geode cluster member which holds a Geode cache. Depending on the topology used it can refer to either a system that responds to client requests or a system that is only a peer to other members.";
+  public static final String TOPIC_GEODE_MANAGER = "Manager";
+  public static final String TOPIC_GEODE_MANAGER__DESC = "The Manager is a member which has the additional role of a managing & monitoring the Geode distributed system.";
+  public static final String TOPIC_GEODE_STATISTICS = "Statistics";
+  public static final String TOPIC_GEODE_STATISTICS__DESC = "iEvery application and server in a Apache Geode distributed system can be configured to perform statistical data collection for analysis.";
+  public static final String TOPIC_GEODE_LIFECYCLE = "Lifecycle";
+  public static final String TOPIC_GEODE_LIFECYCLE__DESC = "Launching, execution and termination of Geode cluster members such as servers and locators.";
+  public static final String TOPIC_GEODE_M_AND_M = "Management-Monitoring";
+  public static final String TOPIC_GEODE_M_AND_M__DESC = "The management of and monitoring of Geode systems using Geode tools, such as Apache Geode Pulse or Data Browser, and JConsole, which is provided with the JDK(TM)";
+  public static final String TOPIC_GEODE_DATA = "Data";
+  public static final String TOPIC_GEODE_DATA__DESC = "User data as stored in regions of the Geode distributed system.";
+  public static final String TOPIC_GEODE_CONFIG = "Configuration";
+  public static final String TOPIC_GEODE_CONFIG__DESC = "Configuration of Apache Geode Cache & Servers/Locators hosting the Cache.";
+  public static final String TOPIC_GEODE_FUNCTION = "Function Execution";
+  public static final String TOPIC_GEODE_FUNCTION__DESC = "The function execution service provides solutions for these application use cases: \n\tAn application that executes a server-side transaction or carries out data updates using the Geode distributed locking service. \n\tAn application that needs to initialize some of its components once on each server, which might be used later by executed functions. Initialization and startup of a third-party service, such as a messaging service. \n\tAny arbitrary aggregation operation that requires iteration over local data sets that can be done more efficiently through a single call to the cache server. \n\tAny kind of external resource provisioning that can be done by executing a function on a server.";
+  public static final String TOPIC_GEODE_HELP = "Help";
+  public static final String TOPIC_GEODE_HELP__DESC = "Provides usage information for gfsh & its commands.";
+  public static final String TOPIC_GEODE_DEBUG_UTIL = "Debug-Utility";
+  public static final String TOPIC_GEODE_DEBUG_UTIL__DESC = "Debugging aids & utilities to use with Apache Geode.";
   public static final String TOPIC_GFSH = "GFSH";
-  public static final String TOPIC_GFSH__DESC = "The GemFire SHell";
+  public static final String TOPIC_GFSH__DESC = "The Geode Shell";
   public static final String TOPIC_SHARED_CONFIGURATION = "Cluster Configuration";
-  public static final String TOPIC_SHARED_CONFIGURATION_HELP = "Configuration for cluster and various groups. It consists of cache.xml, gemfire properties and deployed jars.\nChanges due to gfshs command are persisted to the locator hosting the cluster configuration service.";
+  public static final String TOPIC_SHARED_CONFIGURATION_HELP = "Configuration for cluster and various groups. It consists of cache.xml, geode properties and deployed jars.\nChanges due to gfshs command are persisted to the locator hosting the cluster configuration service.";
   public static final String TOPIC_CHANGELOGLEVEL = "User can change the log-level for a  member run time and generate log contents as per the need";
 
   /*-*************************************************************************
@@ -118,10 +116,10 @@ public class CliStrings {
    * *************************************************************************/
 
   public static final String DESKSTOP_APP_RUN_ERROR_MESSAGE = "Running desktop applications is not supported on %1$s.";
-  public static final String GEMFIRE_HOME_NOT_FOUND_ERROR_MESSAGE = "The GEMFIRE environment variable was not defined.  Please set the GEMFIRE environment variable to the directory where GemFire is installed.";
+  public static final String GEODE_HOME_NOT_FOUND_ERROR_MESSAGE = "The GEODE environment variable was not defined.  Please set the GEODE environment variable to the directory where GEODE is installed.";
   public static final String JAVA_HOME_NOT_FOUND_ERROR_MESSAGE = "Unable to locate the Java executables and dependencies.  Please set the JAVA_HOME environment variable.";
-  public static final String CACHE_XML_NOT_FOUND_MESSAGE = "Warning: The GemFire cache XML file {0} could not be found.";
-  public static final String GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE = "Warning: The GemFire {0}properties file {1} could not be found.";
+  public static final String CACHE_XML_NOT_FOUND_MESSAGE = "Warning: The Geode cache XML file {0} could not be found.";
+  public static final String GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE = "Warning: The Geode {0}properties file {1} could not be found.";
   public static final String MEMBER_NOT_FOUND_ERROR_MESSAGE = "Member {0} could not be found.  Please verify the member name or ID and try again.";
   public static final String NO_MEMBERS_IN_GROUP_ERROR_MESSAGE = "No caching members for group {0} could be found.  Please verify the group and try again.";
   public static final String NO_MEMBERS_FOUND_MESSAGE = "No Members Found";
@@ -132,7 +130,7 @@ public class CliStrings {
   public static final String REGION_NOT_FOUND = "Region : {0} not found";
   public static final String INVALID_REGION_NAME = "Invalid region name";
   public static final String INVALID_FILE_EXTENTION = "Invalid file type, the file extension must be \"{0}\"";
-  public static final String GEMFIRE_DATA_FILE_EXTENSION = ".gfd";
+  public static final String GEODE_DATA_FILE_EXTENSION = ".gfd";
   public static final String LOCATOR_HEADER = "Locator";
   public static final String ERROR__MSG__HEADER = "Error";
   public static final String ZIP_FILE_EXTENSION=".zip";
@@ -372,7 +370,7 @@ public class CliStrings {
   public static final String CONNECT__DEFAULT_SSL_BASE_URL = "https://localhost:" + DistributionConfig.DEFAULT_HTTP_SERVICE_PORT + "/gemfire/v1";
   public static final String CONNECT__URL__HELP = "Indicates the base URL to the Manager's HTTP service.  For example: 'http://<host>:<port>/gemfire/v1' Default is '" + CONNECT__DEFAULT_BASE_URL + "'";
   public static final String CONNECT__USE_HTTP = "use-http";
-  public static final String CONNECT__USE_HTTP__HELP = "Connects to Manager by sending HTTP requests to HTTP service hostint the Management REST API.  You must first 'disconnect' in order to reconnect to the Manager via locator or jmx-manager using JMX.";
+  public static final String CONNECT__USE_HTTP__HELP = "Connects to Manager by sending HTTP requests to HTTP service hosting the Management REST API.  You must first 'disconnect' in order to reconnect to the Manager via locator or jmx-manager using JMX.";
   public static final String CONNECT__USERNAME = "user";
   public static final String CONNECT__USERNAME__HELP = "User name to securely connect to the jmx-manager. If the --password parameter is not specified then it will be prompted for.";
   public static final String CONNECT__PASSWORD = "password";
@@ -401,8 +399,8 @@ public class CliStrings {
   public static final String CONNECT__MSG__ALREADY_CONNECTED = "Already connected to: {0}";
   public static final String CONNECT__MSG__SUCCESS = "Successfully connected to: {0}";
   public static final String CONNECT__MSG__ERROR = "Could not connect to : {0}. {1}";
-  public static final String CONNECT__MSG__SERVICE_UNAVAILABLE_ERROR = "Could not find a GemFire jmx-manager service at {0}.";
-  public static final String CONNECT__MSG__COULD_NOT_CONNECT_TO_LOCATOR_0 = "Could not connect to GemFire Locator service at {0}.";
+  public static final String CONNECT__MSG__SERVICE_UNAVAILABLE_ERROR = "Could not find a Geode jmx-manager service at {0}.";
+  public static final String CONNECT__MSG__COULD_NOT_CONNECT_TO_LOCATOR_0 = "Could not connect to Geode Locator service at {0}.";
   public static final String CONNECT__MSG__COULD_NOT_READ_CONFIG_FROM_0 = "Could not read config from {0}.";
   public static final String CONNECT__MSG__COULD_NOT_CONNECT_TO_LOCATOR_0_POSSIBLY_SSL_CONFIG_ERROR = "Could not connect to Locator at {0}."+Gfsh.LINE_SEPARATOR+"Possible reason: Wrong or no SSL configuration provided.";
   public static final String CONNECT__MSG__COULD_NOT_CONNECT_TO_MANAGER_0_POSSIBLY_SSL_CONFIG_ERROR = "Could not connect to Manager at {0}."+Gfsh.LINE_SEPARATOR+"Possible reason: Wrong or no SSL configuration provided.";
@@ -561,7 +559,7 @@ public class CliStrings {
   public static final String CREATE_REGION__USEATTRIBUTESFROM = "template-region";
   public static final String CREATE_REGION__USEATTRIBUTESFROM__HELP = "Name/Path of the region whose attributes should be duplicated when creating this region.";
   public static final String CREATE_REGION__SKIPIFEXISTS = "skip-if-exists";
-  public static final String CREATE_REGION__SKIPIFEXISTS__HELP = "Skip region ceation if the region already exists.";
+  public static final String CREATE_REGION__SKIPIFEXISTS__HELP = "Skip region creation if the region already exists.";
   public static final String CREATE_REGION__KEYCONSTRAINT = "key-constraint";
   public static final String CREATE_REGION__KEYCONSTRAINT__HELP = "Fully qualified class name of the objects allowed as region keys. Ensures that keys for region entries are all of the same class.";
   public static final String CREATE_REGION__VALUECONSTRAINT = "value-constraint";
@@ -668,7 +666,7 @@ public class CliStrings {
       + CliStrings.LIST_REGION + "\" to verify region exists.";
   public static final String CREATE_REGION__MSG__USE_ATTRIBUTES_FORM_REGIONS_EXISTS_BUT_DIFFERENT_SCOPE_OR_DATAPOLICY_USE_DESCRIBE_REGION_FOR_0 = "The region mentioned for \""
       + CliStrings.CREATE_REGION__USEATTRIBUTESFROM
-      + "\" exists in this GemFire Cluster but with different Scopes or Data Policies on different members. For details, use command \""
+      + "\" exists in this Geode Cluster but with different Scopes or Data Policies on different members. For details, use command \""
       + CliStrings.DESCRIBE_REGION + "\" for \"{0}\".";
   public static final String CREATE_REGION__MSG__USE_ATTRIBUTES_FROM_REGION_0_IS_NOT_WITH_PERSISTENCE = CREATE_REGION__USEATTRIBUTESFROM
       + " region \"{0}\" is not persistent.";
@@ -685,7 +683,7 @@ public class CliStrings {
   public static final String CREATE_REGION__MSG__0_IS_A_PR_CANNOT_HAVE_SUBREGIONS = "\"{0}\" is a Partitioned Region and cannot have Subregions.";
 
   public static final String CREATE_REGION__COMPRESSOR = "compressor";
-  public static final String CREATE_REGION__COMPRESSOR__HELP = "The fully-qualifed class name of the Compressor to use when compressing region entry values.  The default is no compression.";
+  public static final String CREATE_REGION__COMPRESSOR__HELP = "The fully-qualified class name of the Compressor to use when compressing region entry values.  The default is no compression.";
   public static final String CREATE_REGION__MSG__INVALID_COMPRESSOR = "{0} is an invalid Compressor."; // leave space in the end
 
   public static final String CREATE_REGION__OFF_HEAP = "off-heap";
@@ -730,7 +728,7 @@ public class CliStrings {
   public static final String DESCRIBE_DISK_STORE__MEMBER__HELP = "Name/Id of the member with the disk store to be described.";
   public static final String DESCRIBE_DISK_STORE__NAME = "name";
   public static final String DESCRIBE_DISK_STORE__NAME__HELP = "Name of the disk store to be described.";
-  public static final String DESCRIBE_DISK_STORE__ERROR_MESSAGE = "An error occurred while collecting Disk Store information for member (%1$s) with disk store (%2$s) in the GemFire cluster: %3$s";
+  public static final String DESCRIBE_DISK_STORE__ERROR_MESSAGE = "An error occurred while collecting Disk Store information for member (%1$s) with disk store (%2$s) in the Geode cluster: %3$s";
 
   /* 'describe member' command */
   public static final String DESCRIBE_MEMBER = "describe member";
@@ -754,7 +752,7 @@ public class CliStrings {
   
   /* 'export offline-disk-store' command */
   public static final String EXPORT_OFFLINE_DISK_STORE = "export offline-disk-store";
-  public static final String EXPORT_OFFLINE_DISK_STORE__HELP = "Export region data from an offline disk store into gemfire snapshot files.";
+  public static final String EXPORT_OFFLINE_DISK_STORE__HELP = "Export region data from an offline disk store into Geode snapshot files.";
   public static final String EXPORT_OFFLINE_DISK_STORE__DISKSTORENAME = "name";
   public static final String EXPORT_OFFLINE_DISK_STORE__DISKSTORENAME__HELP = "Name of the disk store to be exported.";
   public static final String EXPORT_OFFLINE_DISK_STORE__DIR = "dir";
@@ -834,12 +832,12 @@ public class CliStrings {
   public static final String DESTROY_REGION__REGION__HELP = "Name/Path of the region to be removed.";
 
   public static final String DESTROY_REGION__MSG__REGIONPATH_0_NOT_VALID = "Region path \"{0}\" is not valid.";
-  public static final String DESTROY_REGION__MSG__COULDNOT_FIND_REGIONPATH_0_IN_GEMFIRE = "Could not find a Region with Region path \"{0}\" in this GemFire cluster. If region was recently created, please wait for at least {1} to allow the associated Management resources to be federated.";
+  public static final String DESTROY_REGION__MSG__COULDNOT_FIND_REGIONPATH_0_IN_GEODE = "Could not find a Region with Region path \"{0}\" in this Geode cluster. If region was recently created, please wait for at least {1} to allow the associated Management resources to be federated.";
   public static final String DESTROY_REGION__MSG__AND_ITS_SUBREGIONS = "and its subregions";
   public static final String DESTROY_REGION__MSG__REGION_0_1_DESTROYED = "\"{0}\" {1} destroyed successfully.";
   public static final String DESTROY_REGION__MSG__ERROR_OCCURRED_WHILE_DESTROYING_0_REASON_1 = "Error occurred while destroying region \"{0}\". Reason: {1}";
   public static final String DESTROY_REGION__MSG__UNKNOWN_RESULT_WHILE_DESTROYING_REGION_0_REASON_1 = "Unknown result while destroying region \"{0}\". Reason: {1}";
-  public static final String DESTROY_REGION__MSG__COULDNOT_FIND_MEMBER_WITH_REGION_0 = "Could not find a GemFire member which hosts a region with Region path \"{0}\"";
+  public static final String DESTROY_REGION__MSG__COULDNOT_FIND_MEMBER_WITH_REGION_0 = "Could not find a Geode member which hosts a region with Region path \"{0}\"";
   public static final String DESTROY_REGION__MSG__SPECIFY_REGIONPATH_TO_DESTROY = "Please specify region path for the region to be destroyed.";
   public static final String DESTROY_REGION__MSG__ERROR_WHILE_DESTROYING_REGION_0_REASON_1 = "Error while destroying region {0}. Reason: {1}";
 
@@ -1139,7 +1137,7 @@ public class CliStrings {
   /* list index */
   public static final String LIST_INDEX = "list indexes";
   public static final String LIST_INDEX__HELP = "Display the list of indexes created for all members.";
-  public static final String LIST_INDEX__ERROR_MESSAGE = "An error occurred while collecting all Index information across the GemFire cluster: %1$s";
+  public static final String LIST_INDEX__ERROR_MESSAGE = "An error occurred while collecting all Index information across the Geode cluster: %1$s";
   public static final String LIST_INDEX__INDEXES_NOT_FOUND_MESSAGE = "No Indexes Found";
   public static final String LIST_INDEX__STATS = "with-stats";
   public static final String LIST_INDEX__STATS__HELP = "Whether statistics should also be displayed.";
@@ -1147,7 +1145,7 @@ public class CliStrings {
   /* list disk-store command */
   public static final String LIST_DISK_STORE = "list disk-stores";
   public static final String LIST_DISK_STORE__HELP = "Display disk stores for all members.";
-  public static final String LIST_DISK_STORE__ERROR_MESSAGE = "An error occurred while collecting Disk Store information for all members across the GemFire cluster: %1$s";
+  public static final String LIST_DISK_STORE__ERROR_MESSAGE = "An error occurred while collecting Disk Store information for all members across the Geode cluster: %1$s";
   public static final String LIST_DISK_STORE__DISK_STORES_NOT_FOUND_MESSAGE = "No Disk Stores Found";
 
   /* 'list member' command */
@@ -1262,7 +1260,7 @@ public class CliStrings {
   public static final String QUERY__MSG__INVALID_QUERY = "Query is invalid due for error : <{0}>";
   public static final String QUERY__MSG__REGIONS_NOT_FOUND = "Cannot find regions <{0}> in any of the members";
   public static final String QUERY__MSG__NOT_SUPPORTED_ON_MEMBERS = CliStrings.QUERY
-      + " command should be used only from shell. Use QueryService API for running query inside Gemfire VMs";
+      + " command should be used only from shell. Use QueryService API for running query inside Geode VMs";
 
   /* 'rebalance' command */
   public static final String REBALANCE = "rebalance";
@@ -1387,7 +1385,7 @@ public class CliStrings {
 
   /* show dead-lock command */
   public static final String SHOW_DEADLOCK = "show dead-locks";
-  public static final String SHOW_DEADLOCK__HELP = "Display any deadlocks in the GemFire distributed system.";
+  public static final String SHOW_DEADLOCK__HELP = "Display any deadlocks in the Geode distributed system.";
   public static final String SHOW_DEADLOCK__DEPENDENCIES__FILE = "file";
   public static final String SHOW_DEADLOCK__DEPENDENCIES__FILE__HELP = "Name of the file to which dependencies between members will be written.";
   public static final String SHOW_DEADLOCK__NO__DEADLOCK = "No deadlock was detected.";
@@ -1569,18 +1567,18 @@ public class CliStrings {
 
   /* start gfmon command */
   public static final String START_PULSE = "start pulse";
-  public static final String START_PULSE__ERROR = "An error occurred while launching Gemfire Pulse - %1$s";
+  public static final String START_PULSE__ERROR = "An error occurred while launching Geode Pulse - %1$s";
   public static final String START_PULSE__HELP = "Open a new window in the default Web browser with the URL for the Pulse application.";
-  public static final String START_PULSE__RUN = "Launched GemFire Pulse";
+  public static final String START_PULSE__RUN = "Launched Geode Pulse";
   public static final String START_PULSE__URL = "url";
   public static final String START_PULSE__URL__HELP = "URL of the Pulse Web application.";
-  public static final String START_PULSE__URL__NOTFOUND = "Could not find the URL for GemFire Pulse.";
+  public static final String START_PULSE__URL__NOTFOUND = "Could not find the URL for Geode Pulse.";
 
   /* 'start jsonsole' command */
   public static final String START_JCONSOLE = "start jconsole";
-  public static final String START_JCONSOLE__HELP = "Start the JDK's JConsole tool in a separate process. JConsole will be launched, but connecting to GemFire must be done manually.";
+  public static final String START_JCONSOLE__HELP = "Start the JDK's JConsole tool in a separate process. JConsole will be launched, but connecting to Geode must be done manually.";
   public static final String START_JCONSOLE__CATCH_ALL_ERROR_MESSAGE = "An error occurred while launching JConsole = %1$s";
-  public static final String START_JCONSOLE__CONNECT_BY_MEMBER_NAME_ID_ERROR_MESSAGE = "Connecting by the GemFire member's name or ID is not currently supported.\nPlease specify the member as '<hostname|IP>[PORT].";
+  public static final String START_JCONSOLE__CONNECT_BY_MEMBER_NAME_ID_ERROR_MESSAGE = "Connecting by the Geode member's name or ID is not currently supported.\nPlease specify the member as '<hostname|IP>[PORT].";
   public static final String START_JCONSOLE__INTERVAL = "interval";
   public static final String START_JCONSOLE__INTERVAL__HELP = "Update internal (in seconds). This parameter is passed as -interval to JConsole.";
   public static final String START_JCONSOLE__IO_EXCEPTION_MESSAGE = "An IO error occurred while launching JConsole.\nPlease ensure that JAVA_HOME is set to the JDK installation or the JDK bin directory is in the system PATH.";
@@ -1599,7 +1597,7 @@ public class CliStrings {
 
   /* 'start jvisualvm command' */
   public static final String START_JVISUALVM = "start jvisualvm";
-  public static final String START_JVISUALVM__HELP = "Start the JDK's Java VisualVM (jvisualvm) tool in a separate process. Java VisualVM will be launched, but connecting to GemFire must be done manually.";
+  public static final String START_JVISUALVM__HELP = "Start the JDK's Java VisualVM (jvisualvm) tool in a separate process. Java VisualVM will be launched, but connecting to Geode must be done manually.";
   public static final String START_JVISUALVM__ERROR_MESSAGE = "An error occurred while launching Java VisualVM - %1$s";
   public static final String START_JVISUALVM__EXPECTED_JDK_VERSION_ERROR_MESSAGE = "Java VisualVM was not bundled with the JDK until version 1.6.\nDownload and install Java VisualVM to the JDK bin directory separately.";
   public static final String START_JVISUALVM__J = "J";
@@ -1625,19 +1623,19 @@ public class CliStrings {
   public static final String START_LOCATOR__INCLUDE_SYSTEM_CLASSPATH = "include-system-classpath";
   public static final String START_LOCATOR__INCLUDE_SYSTEM_CLASSPATH__HELP = "Includes the System CLASSPATH on the Locator's CLASSPATH. The System CLASSPATH is not included by default.";
   public static final String START_LOCATOR__LOCATORS = LOCATORS;
-  public static final String START_LOCATOR__LOCATORS__HELP = "Sets the list of Locators used by this Locator to join the appropriate GemFire cluster.";
+  public static final String START_LOCATOR__LOCATORS__HELP = "Sets the list of Locators used by this Locator to join the appropriate Geode cluster.";
   public static final String START_LOCATOR__LOG_LEVEL = LOG_LEVEL;
   public static final String START_LOCATOR__LOG_LEVEL__HELP = "Sets the level of output logged to the Locator log file.  Possible values for log-level include: finest, finer, fine, config, info, warning, severe, none.";
   public static final String START_LOCATOR__MCAST_ADDRESS = MCAST_ADDRESS;
-  public static final String START_LOCATOR__MCAST_ADDRESS__HELP = "The IP address or hostname used to bind the UPD socket for multi-cast networking so the Locator can communicate other members in the GemFire cluster using a common multicast address and port.  If mcast-port is zero, then mcast-address is ignored.";
+  public static final String START_LOCATOR__MCAST_ADDRESS__HELP = "The IP address or hostname used to bind the UPD socket for multi-cast networking so the Locator can communicate other members in the Geode cluster using a common multicast address and port.  If mcast-port is zero, then mcast-address is ignored.";
   public static final String START_LOCATOR__MCAST_PORT = MCAST_PORT;
-  public static final String START_LOCATOR__MCAST_PORT__HELP = "Sets the port used for multi-cast networking so the Locator can communicate with other members of the GemFire cluster.  A zero value disables mcast.";
+  public static final String START_LOCATOR__MCAST_PORT__HELP = "Sets the port used for multi-cast networking so the Locator can communicate with other members of the Geode cluster.  A zero value disables mcast.";
   public static final String START_LOCATOR__MEMBER_NAME = "name";
-  public static final String START_LOCATOR__MEMBER_NAME__HELP = "The member name to give this Locator in the GemFire cluster.";
+  public static final String START_LOCATOR__MEMBER_NAME__HELP = "The member name to give this Locator in the Geode cluster.";
   public static final String START_LOCATOR__PORT = "port";
   public static final String START_LOCATOR__PORT__HELP = "Port the Locator will listen on.";
   public static final String START_LOCATOR__PROPERTIES = "properties-file";
-  public static final String START_LOCATOR__PROPERTIES__HELP = "The gemfire.properties file for configuring the Locator's distributed system. The file's path can be absolute or relative to the gfsh working directory (--dir=).";
+  public static final String START_LOCATOR__PROPERTIES__HELP = "The geode.properties file for configuring the Locator's distributed system. The file's path can be absolute or relative to the gfsh working directory (--dir=).";
   public static final String START_LOCATOR__SECURITY_PROPERTIES = "security-properties-file";
   public static final String START_LOCATOR__SECURITY_PROPERTIES__HELP = "The gfsecurity.properties file for configuring the Locator's security configuration in the distributed system. The file's path can be absolute or relative to gfsh directory (--dir=).";
   public static final String START_LOCATOR__INITIALHEAP = "initial-heap";
@@ -1648,7 +1646,7 @@ public class CliStrings {
   public static final String START_LOCATOR__MAXHEAP__HELP = "Maximum size of the heap in the same format as the JVM -Xmx parameter.";
   public static final String START_LOCATOR__GENERAL_ERROR_MESSAGE = "An error occurred while attempting to start a Locator in %1$s on %2$s: %3$s";
   public static final String START_LOCATOR__PROCESS_TERMINATED_ABNORMALLY_ERROR_MESSAGE = "The Locator process terminated unexpectedly with exit status %1$d. Please refer to the log file in %2$s for full details.%n%n%3$s";
-  public static final String START_LOCATOR__RUN_MESSAGE = "Starting a GemFire Locator in %1$s...";
+  public static final String START_LOCATOR__RUN_MESSAGE = "Starting a Geode Locator in %1$s...";
   public static final String START_LOCATOR__MSG__COULD_NOT_CREATE_DIRECTORY_0_VERIFY_PERMISSIONS = "Could not create directory {0}. Please verify directory path or user permissions.";
   public static final String START_LOCATOR__CONNECT = "connect";
   public static final String START_LOCATOR__CONNECT__HELP = "When connect is set to false , Gfsh does not automatically connect to the locator which is started using this command.";
@@ -1679,14 +1677,14 @@ public class CliStrings {
   public static final String START_MANAGER__MAXHEAP__HELP = "Maximum size of the heap in the same format as the JVM -Xmx parameter.";
   public static final String START_MANAGER__INITIALHEAP = "initial-heap";
   public static final String START_MANAGER__INITIALHEAP__HELP = "Initial size of the heap in the same format as the JVM -Xms parameter.";
-  public static final String START_MANAGER__GEMFIREPROPS = "G";
-  public static final String START_MANAGER__GEMFIREPROPS__HELP = "GemFire property passed as a <name>=<value> pair.";
+  public static final String START_MANAGER__GEODEPROPS = "G";
+  public static final String START_MANAGER__GEODEPROPS__HELP = "Geode property passed as a <name>=<value> pair.";
   public static final String START_MANAGER__J = "J";
   public static final String START_MANAGER__J__HELP = "Argument passed to the JVM on which the Locator will run. For example, --J=-Dfoo.bar=true will set the property \"foo.bar\" to \"true\".";
 
   /* 'start server' command */
   public static final String START_SERVER = "start server";
-  public static final String START_SERVER__HELP = "Start a GemFire Cache Server.";
+  public static final String START_SERVER__HELP = "Start a Geode Cache Server.";
   public static final String START_SERVER__ASSIGN_BUCKETS = "assign-buckets";
   public static final String START_SERVER__ASSIGN_BUCKETS__HELP = "Whether to assign buckets to the partitioned regions of the cache on server start.";
   public static final String START_SERVER__BIND_ADDRESS = "bind-address";
@@ -1702,7 +1700,7 @@ public class CliStrings {
   public static final String START_SERVER__DISABLE_EXIT_WHEN_OUT_OF_MEMORY = "disable-exit-when-out-of-memory";
   public static final String START_SERVER__DISABLE_EXIT_WHEN_OUT_OF_MEMORY_HELP = "Prevents the JVM from exiting when an OutOfMemoryError occurs.";
   public static final String START_SERVER__ENABLE_TIME_STATISTICS = ENABLE_TIME_STATISTICS;
-  public static final String START_SERVER__ENABLE_TIME_STATISTICS__HELP = "Causes additional time-based statistics to be gathered for GemFire operations.";
+  public static final String START_SERVER__ENABLE_TIME_STATISTICS__HELP = "Causes additional time-based statistics to be gathered for Geode operations.";
   public static final String START_SERVER__FORCE = "force";
   public static final String START_SERVER__FORCE__HELP = "Whether to allow the PID file from a previous Cache Server run to be overwritten.";
   public static final String START_SERVER__GROUP = "group";
@@ -1714,9 +1712,9 @@ public class CliStrings {
   public static final String START_SERVER__J = "J";
   public static final String START_SERVER__J__HELP = "Argument passed to the JVM on which the server will run. For example, --J=-Dfoo.bar=true will set the system property \"foo.bar\" to \"true\".";
   public static final String START_SERVER__LOCATORS = LOCATORS;
-  public static final String START_SERVER__LOCATORS__HELP = "Sets the list of Locators used by the Cache Server to join the appropriate GemFire cluster.";
+  public static final String START_SERVER__LOCATORS__HELP = "Sets the list of Locators used by the Cache Server to join the appropriate Geode cluster.";
   public static final String START_SERVER__LOCK_MEMORY = ConfigurationProperties.LOCK_MEMORY;
-  public static final String START_SERVER__LOCK_MEMORY__HELP = "Causes GemFire to lock heap and off-heap memory pages into RAM. This prevents the operating system from swapping the pages out to disk, which can cause severe performance degradation. When you use this option, also configure the operating system limits for locked memory.";
+  public static final String START_SERVER__LOCK_MEMORY__HELP = "Causes Geode to lock heap and off-heap memory pages into RAM. This prevents the operating system from swapping the pages out to disk, which can cause severe performance degradation. When you use this option, also configure the operating system limits for locked memory.";
   public static final String START_SERVER__LOCATOR_WAIT_TIME = "locator-wait-time";
   public static final String START_SERVER__LOCATOR_WAIT_TIME_HELP = "Sets the number of seconds the server will wait for a locator to become available during startup before giving up.";
   public static final String START_SERVER__LOG_LEVEL = LOG_LEVEL;
@@ -1724,38 +1722,38 @@ public class CliStrings {
   public static final String START_SERVER__MAXHEAP = "max-heap";
   public static final String START_SERVER__MAXHEAP__HELP = "Maximum size of the heap in the same format as the JVM -Xmx parameter.";
   public static final String START_SERVER__MCAST_ADDRESS = MCAST_ADDRESS;
-  public static final String START_SERVER__MCAST_ADDRESS__HELP = "The IP address or hostname used to bind the UPD socket for multi-cast networking so the Cache Server can communicate with other members in the GemFire cluster.  If mcast-port is zero, then mcast-address is ignored.";
+  public static final String START_SERVER__MCAST_ADDRESS__HELP = "The IP address or hostname used to bind the UPD socket for multi-cast networking so the Cache Server can communicate with other members in the Geode cluster.  If mcast-port is zero, then mcast-address is ignored.";
   public static final String START_SERVER__MCAST_PORT = MCAST_PORT;
-  public static final String START_SERVER__MCAST_PORT__HELP = "Sets the port used for multi-cast networking so the Cache Server can communicate with other members of the GemFire cluster.  A zero value disables mcast.";
+  public static final String START_SERVER__MCAST_PORT__HELP = "Sets the port used for multi-cast networking so the Cache Server can communicate with other members of the Geode cluster.  A zero value disables mcast.";
   public static final String START_SERVER__NAME = "name";
-  public static final String START_SERVER__NAME__HELP = "The member name to give this Cache Server in the GemFire cluster.";
+  public static final String START_SERVER__NAME__HELP = "The member name to give this Cache Server in the Geode cluster.";
   public static final String START_SERVER__MEMCACHED_PORT = MEMCACHED_PORT;
-  public static final String START_SERVER__MEMCACHED_PORT__HELP = "Sets the port that the GemFire memcached service listens on for memcached clients.";
+  public static final String START_SERVER__MEMCACHED_PORT__HELP = "Sets the port that the Geode memcached service listens on for memcached clients.";
   public static final String START_SERVER__MEMCACHED_PROTOCOL = MEMCACHED_PROTOCOL;
-  public static final String START_SERVER__MEMCACHED_PROTOCOL__HELP = "Sets the protocol that the GemFire memcached service uses (ASCII or BINARY).";
+  public static final String START_SERVER__MEMCACHED_PROTOCOL__HELP = "Sets the protocol that the Geode memcached service uses (ASCII or BINARY).";
   public static final String START_SERVER__MEMCACHED_BIND_ADDRESS = MEMCACHED_BIND_ADDRESS;
-  public static final String START_SERVER__MEMCACHED_BIND_ADDRESS__HELP = "Sets the IP address the GemFire memcached service listens on for memcached clients. The default is to bind to the first non-loopback address for this machine.";
+  public static final String START_SERVER__MEMCACHED_BIND_ADDRESS__HELP = "Sets the IP address the Geode memcached service listens on for memcached clients. The default is to bind to the first non-loopback address for this machine.";
   public static final String START_SERVER__OFF_HEAP_MEMORY_SIZE = ConfigurationProperties.OFF_HEAP_MEMORY_SIZE;
   public static final String START_SERVER__OFF_HEAP_MEMORY_SIZE__HELP = "The total size of off-heap memory specified as off-heap-memory-size=<n>[g|m]. <n> is the size. [g|m] indicates whether the size should be interpreted as gigabytes or megabytes. A non-zero size causes that much memory to be allocated from the operating system and reserved for off-heap use.";
   public static final String START_SERVER__PROPERTIES = "properties-file";
-  public static final String START_SERVER__PROPERTIES__HELP = "The gemfire.properties file for configuring the Cache Server's distributed system. The file's path can be absolute or relative to the gfsh working directory.";
+  public static final String START_SERVER__PROPERTIES__HELP = "The geode.properties file for configuring the Cache Server's distributed system. The file's path can be absolute or relative to the gfsh working directory.";
   public static final String START_SERVER__REDIS_PORT = ConfigurationProperties.REDIS_PORT;
-  public static final String START_SERVER__REDIS_PORT__HELP = "Sets the port that the GemFire Redis service listens on for Redis clients.";
+  public static final String START_SERVER__REDIS_PORT__HELP = "Sets the port that the Geode Redis service listens on for Redis clients.";
   public static final String START_SERVER__REDIS_BIND_ADDRESS = ConfigurationProperties.REDIS_BIND_ADDRESS;
-  public static final String START_SERVER__REDIS_BIND_ADDRESS__HELP = "Sets the IP address the GemFire Redis service listens on for Redis clients. The default is to bind to the first non-loopback address for this machine.";
+  public static final String START_SERVER__REDIS_BIND_ADDRESS__HELP = "Sets the IP address the Geode Redis service listens on for Redis clients. The default is to bind to the first non-loopback address for this machine.";
   public static final String START_SERVER__REDIS_PASSWORD = ConfigurationProperties.REDIS_PASSWORD;
-  public static final String START_SERVER__REDIS_PASSWORD__HELP = "Sets the authentication password for GemFireRedisServer";
+  public static final String START_SERVER__REDIS_PASSWORD__HELP = "Sets the authentication password for GeodeRedisServer";
   public static final String START_SERVER__SECURITY_PROPERTIES = "security-properties-file";
   public static final String START_SERVER__SECURITY_PROPERTIES__HELP = "The gfsecurity.properties file for configuring the Server's security configuration in the distributed system. The file's path can be absolute or relative to gfsh directory.";
   public static final String START_SERVER__REBALANCE = "rebalance";
-  public static final String START_SERVER__REBALANCE__HELP = "Whether to initiate rebalancing across the GemFire cluster.";
+  public static final String START_SERVER__REBALANCE__HELP = "Whether to initiate rebalancing across the Geode cluster.";
   public static final String START_SERVER__SERVER_BIND_ADDRESS = SERVER_BIND_ADDRESS;
   public static final String START_SERVER__SERVER_BIND_ADDRESS__HELP = "The IP address that this distributed system's server sockets in a client-server topology will be bound. If set to an empty string then all of the local machine's addresses will be listened on.";
   public static final String START_SERVER__SERVER_PORT = "server-port";
   public static final String START_SERVER__SERVER_PORT__HELP = "The port that the distributed system's server sockets in a client-server topology will listen on.  The default server-port is "
       + CacheServer.DEFAULT_PORT + ".";
   public static final String START_SERVER__SPRING_XML_LOCATION = "spring-xml-location";
-  public static final String START_SERVER__SPRING_XML_LOCATION_HELP = "Specifies the location of a Spring XML configuration file(s) for bootstrapping and configuring a GemFire Server.";
+  public static final String START_SERVER__SPRING_XML_LOCATION_HELP = "Specifies the location of a Spring XML configuration file(s) for bootstrapping and configuring a Geode Server.";
   public static final String START_SERVER__STATISTIC_ARCHIVE_FILE = STATISTIC_ARCHIVE_FILE;
   public static final String START_SERVER__STATISTIC_ARCHIVE_FILE__HELP = "The file that statistic samples are written to.  An empty string (default) disables statistic archival.";
   // public static final String START_SERVER__START_LOCATOR = "start-locator";
@@ -1763,9 +1761,9 @@ public class CliStrings {
   // "To start embedded Locator with given endpoints in the format: host[port]. If no endpoints are given defaults (localhost[10334]) are assumed.";
   public static final String START_SERVER__USE_CLUSTER_CONFIGURATION = USE_CLUSTER_CONFIGURATION;
   public static final String START_SERVER__USE_CLUSTER_CONFIGURATION__HELP = "When set to true, the server requests the configuration from locator's cluster configuration service.";
-  public static final String START_SERVER__GENERAL_ERROR_MESSAGE = "An error occurred while attempting to start a GemFire Cache Server: %1$s";
+  public static final String START_SERVER__GENERAL_ERROR_MESSAGE = "An error occurred while attempting to start a Geode Cache Server: %1$s";
   public static final String START_SERVER__PROCESS_TERMINATED_ABNORMALLY_ERROR_MESSAGE = "The Cache Server process terminated unexpectedly with exit status %1$d. Please refer to the log file in %2$s for full details.%n%n%3$s";
-  public static final String START_SERVER__RUN_MESSAGE = "Starting a GemFire Server in %1$s...";
+  public static final String START_SERVER__RUN_MESSAGE = "Starting a Geode Server in %1$s...";
   public static final String START_SERVER__MSG__COULD_NOT_CREATE_DIRECTORY_0_VERIFY_PERMISSIONS = "Could not create directory {0}. Please verify directory path or user permissions.";
   
   public static final String START_SERVER__CRITICAL__HEAP__PERCENTAGE = "critical-heap-percentage";
@@ -1826,14 +1824,14 @@ public class CliStrings {
   public static final String START_VSD__FILE__HELP = "File or directory from which to read the statistics archive(s).";
   public static final String START_VSD__ERROR_MESSAGE = "An error occurred while launching VSD - %1$s";
   public static final String START_VSD__HELP = "Start VSD in a separate process.";
-  public static final String START_VSD__NOT_FOUND_ERROR_MESSAGE = "The location of VSD could not be found.  Please ensure VSD was properly installed under GemFire home (%1$s).";
-  public static final String START_VSD__RUN = "Launched GemFire Visual Statistics Display (VSD) (see GemFire log files for issues on start)";
+  public static final String START_VSD__NOT_FOUND_ERROR_MESSAGE = "The location of VSD could not be found.  Please ensure VSD was properly installed under Geode home (%1$s).";
+  public static final String START_VSD__RUN = "Launched Geode Visual Statistics Display (VSD) (see Geode log files for issues on start)";
 
   /* start databrowser command */
   public static final String START_DATABROWSER = "start data-browser";
   public static final String START_DATABROWSER__HELP = "Start Data Browser in a separate process.";
-  public static final String START_DATABROWSER__NOT_FOUND_ERROR_MESSAGE = "The location of DataBrowser could not be found.  Please ensure DataBrowser was properly installed under GemFire home (%1$s).";
-  public static final String START_DATABROWSER__RUN = "Launched GemFire DataBrowser (see GemFire log files for issues on start)";
+  public static final String START_DATABROWSER__NOT_FOUND_ERROR_MESSAGE = "The location of DataBrowser could not be found.  Please ensure DataBrowser was properly installed under Geode home (%1$s).";
+  public static final String START_DATABROWSER__RUN = "Launched Geode DataBrowser (see Geode log files for issues on start)";
   public static final String START_DATABROWSER__ERROR = "An error occurred while launching DataBrowser - %1$s";
 
   /* status gateway-receiver */
@@ -1862,7 +1860,7 @@ public class CliStrings {
   public static final String STATUS_LOCATOR__HOST = "host";
   public static final String STATUS_LOCATOR__HOST__HELP = "Hostname or IP address on which the Locator is running.";
   public static final String STATUS_LOCATOR__MEMBER = "name";
-  public static final String STATUS_LOCATOR__MEMBER__HELP = "Member name or ID of the Locator in the GemFire cluster.";
+  public static final String STATUS_LOCATOR__MEMBER__HELP = "Member name or ID of the Locator in the Geode cluster.";
   public static final String STATUS_LOCATOR__PID = "pid";
   public static final String STATUS_LOCATOR__PID__HELP = "Process ID (PID) of the running Locator.";
   public static final String STATUS_LOCATOR__PORT = "port";
@@ -1872,15 +1870,15 @@ public class CliStrings {
 
   /* 'status server' command */
   public static final String STATUS_SERVER = "status server";
-  public static final String STATUS_SERVER__HELP = "Display the status of a GemFire Cache Server.";
+  public static final String STATUS_SERVER__HELP = "Display the status of a Geode Cache Server.";
   public static final String STATUS_SERVER__DIR = "dir";
   public static final String STATUS_SERVER__DIR__HELP = "Working directory in which the Cache Server is running. The default is the current directory.";
-  public static final String STATUS_SERVER__GENERAL_ERROR_MESSAGE = "An error occurred while attempting to determine the status of GemFire Cache server: %1$s";
+  public static final String STATUS_SERVER__GENERAL_ERROR_MESSAGE = "An error occurred while attempting to determine the status of Geode Cache server: %1$s";
   public static final String STATUS_SERVER__MEMBER = "name";
-  public static final String STATUS_SERVER__MEMBER__HELP = "Member name or ID of the Cache Server in the GemFire cluster.";
-  public static final String STATUS_SERVER__NO_SERVER_FOUND_FOR_MEMBER_ERROR_MESSAGE = "No GemFire Cache Server with member name or ID {0} could be found.";
+  public static final String STATUS_SERVER__MEMBER__HELP = "Member name or ID of the Cache Server in the Geode cluster.";
+  public static final String STATUS_SERVER__NO_SERVER_FOUND_FOR_MEMBER_ERROR_MESSAGE = "No Geode Cache Server with member name or ID {0} could be found.";
   public static final String STATUS_SERVER__PID = "pid";
-  public static final String STATUS_SERVER__PID__HELP = "Process ID (PID) of the running GemFire Cache Server.";
+  public static final String STATUS_SERVER__PID__HELP = "Process ID (PID) of the running Geode Cache Server.";
 
   /* stop gateway-reciver */
   public static final String STOP_GATEWAYRECEIVER = "stop gateway-receiver";
@@ -1906,28 +1904,28 @@ public class CliStrings {
   public static final String STOP_LOCATOR__DIR = "dir";
   public static final String STOP_LOCATOR__DIR__HELP = "Working directory in which the Locator is running. The default is the current directory.";
   public static final String STOP_LOCATOR__MEMBER = "name";
-  public static final String STOP_LOCATOR__MEMBER__HELP = "Member name or ID of the Locator in the GemFire cluster.";
+  public static final String STOP_LOCATOR__MEMBER__HELP = "Member name or ID of the Locator in the Geode cluster.";
   public static final String STOP_LOCATOR__PID = "pid";
   public static final String STOP_LOCATOR__PID__HELP = "The process id (PID) of the running Locator.";
   public static final String STOP_LOCATOR__GENERAL_ERROR_MESSAGE = "An error occurred while attempting to stop a Locator: %1$s";
   public static final String STOP_LOCATOR__LOCATOR_IS_CACHE_SERVER_ERROR_MESSAGE = "The Locator identified by {0} is also a cache server and cannot be shutdown using 'stop locator'.  Please use 'stop server' instead.";
   public static final String STOP_LOCATOR__NO_LOCATOR_FOUND_FOR_MEMBER_ERROR_MESSAGE = "No Locator with member name or ID {0} could be found.";
-  public static final String STOP_LOCATOR__NOT_LOCATOR_ERROR_MESSAGE = "The GemFire member identified by {0} is not a Locator and cannot be shutdown using 'stop locator'.";
+  public static final String STOP_LOCATOR__NOT_LOCATOR_ERROR_MESSAGE = "The Geode member identified by {0} is not a Locator and cannot be shutdown using 'stop locator'.";
   public static final String STOP_LOCATOR__SHUTDOWN_MEMBER_MESSAGE = "Locator {0} has been requested to stop.";
   public static final String STOP_LOCATOR__STOPPING_LOCATOR_MESSAGE = "Stopping Locator running in %1$s on %2$s as %3$s...%nProcess ID: %4$d%nLog File: %5$s";
 
   /* 'stop server' command */
   public static final String STOP_SERVER = "stop server";
-  public static final String STOP_SERVER__HELP = "Stop a GemFire Cache Server.";
+  public static final String STOP_SERVER__HELP = "Stop a Geode Cache Server.";
   public static final String STOP_SERVER__DIR = "dir";
   public static final String STOP_SERVER__DIR__HELP = "Working directory in which the Cache Server is running. The default is the current directory.";
   public static final String STOP_SERVER__GENERAL_ERROR_MESSAGE = "An error occurred while attempting to stop a Cache Server: %1$s";
-  public static final String STOP_SERVER__MEMBER_IS_NOT_SERVER_ERROR_MESSAGE = "Attempting to stop a GemFire member that is not a Cache Server using 'stop server'; the operation is not permitted.";
+  public static final String STOP_SERVER__MEMBER_IS_NOT_SERVER_ERROR_MESSAGE = "Attempting to stop a Geode member that is not a Cache Server using 'stop server'; the operation is not permitted.";
   public static final String STOP_SERVER__MEMBER = "name";
-  public static final String STOP_SERVER__MEMBER__HELP = "Member name or ID of the Cache Server in the GemFire cluster.";
+  public static final String STOP_SERVER__MEMBER__HELP = "Member name or ID of the Cache Server in the Geode cluster.";
   public static final String STOP_SERVER__NO_SERVER_FOUND_FOR_MEMBER_ERROR_MESSAGE = "No Cache Server with member name or ID {0} could be found.";
   public static final String STOP_SERVER__PID = "pid";
-  public static final String STOP_SERVER__PID__HELP = "Process ID (PID) of the running GemFire Cache Server.";
+  public static final String STOP_SERVER__PID__HELP = "Process ID (PID) of the running Geode Cache Server.";
   public static final String STOP_SERVER__SHUTDOWN_MEMBER_MESSAGE = "Cache Server {0} has been requested to stop.";
   public static final String STOP_SERVER__STOPPING_SERVER_MESSAGE = "Stopping Cache Server running in %1$s on %2$s as %3$s...%nProcess ID: %4$d%nLog File: %5$s";
 
@@ -2151,7 +2149,7 @@ public class CliStrings {
   
   
   public static final String CONFIGURE_PDX = "configure pdx";
-  public static final String CONFIGURE_PDX__HELP = "Configures GemFire's Portable Data eXchange for all the cache(s) in the cluster. This command would not take effect on the running members in the system.\n This command persists the pdx configuration in the locator with cluster configuration service. \n This command should be issued before starting any data members.";
+  public static final String CONFIGURE_PDX__HELP = "Configures Geode's Portable Data eXchange for all the cache(s) in the cluster. This command would not take effect on the running members in the system.\n This command persists the pdx configuration in the locator with cluster configuration service. \n This command should be issued before starting any data members.";
   public static final String CONFIGURE_PDX__READ__SERIALIZED = "read-serialized";
   public static final String CONFIGURE_PDX__READ__SERIALIZED__HELP = "Set to true to have PDX deserialization produce a PdxInstance instead of an instance of the domain class";
   public static final String CONFIGURE_PDX__IGNORE__UNREAD_FIELDS = "ignore-unread-fields";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f606706e/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java
index fc366ce..a67cae3 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/JmxOperationInvoker.java
@@ -184,7 +184,7 @@ public class JmxOperationInvoker implements OperationInvoker {
     if (!StringUtils.isBlank(gfSecurityPropertiesPathToUse)) {
       // User specified gfSecurity properties doesn't exist
       if (!IOUtils.isExistingPathname(gfSecurityPropertiesPathToUse)) {
-        gfshInstance.printAsSevere(CliStrings.format(CliStrings.GEMFIRE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, "Security ", gfSecurityPropertiesPathToUse));
+        gfshInstance.printAsSevere(CliStrings.format(CliStrings.GEODE_0_PROPERTIES_1_NOT_FOUND_MESSAGE, "Security ", gfSecurityPropertiesPathToUse));
       } else {
         gfSecurityPropertiesUrl = new File(gfSecurityPropertiesPathToUse).toURI().toURL();
       }


[10/16] incubator-geode git commit: GEODE-1349: prevent 'java.lang.InternalError: MemoryPool not found'

Posted by kl...@apache.org.
GEODE-1349: prevent 'java.lang.InternalError: MemoryPool not found'


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

Branch: refs/heads/feature/GEODE-1565
Commit: bcffff9f0cc11c7e6d1055c6c72cf58383eca8b7
Parents: 374a1dc
Author: Jens Deppe <jd...@pivotal.io>
Authored: Wed May 25 13:45:00 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Jun 17 13:19:47 2016 -0700

----------------------------------------------------------------------
 .../gemfire/internal/stats50/VMStats50.java     | 46 +++++++++++++++++---
 1 file changed, 41 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bcffff9f/geode-core/src/main/java/com/gemstone/gemfire/internal/stats50/VMStats50.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/stats50/VMStats50.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/stats50/VMStats50.java
index 53d8f0e..6f2eea6 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/stats50/VMStats50.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/stats50/VMStats50.java
@@ -16,21 +16,44 @@
  */
 package com.gemstone.gemfire.internal.stats50;
 
-import com.gemstone.gemfire.*;
+import java.lang.management.ClassLoadingMXBean;
+import java.lang.management.GarbageCollectorMXBean;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MemoryMXBean;
+import java.lang.management.MemoryPoolMXBean;
+import java.lang.management.MemoryUsage;
+import java.lang.management.OperatingSystemMXBean;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.StatisticDescriptor;
+import com.gemstone.gemfire.Statistics;
+import com.gemstone.gemfire.StatisticsFactory;
+import com.gemstone.gemfire.StatisticsType;
+import com.gemstone.gemfire.StatisticsTypeFactory;
+import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.ClassPathLoader;
 import com.gemstone.gemfire.internal.StatisticsTypeFactoryImpl;
 import com.gemstone.gemfire.internal.VMStatsContract;
-
-import java.lang.management.*;
-import java.lang.reflect.Method;
-import java.util.*;
+import com.gemstone.gemfire.internal.logging.LogService;
 
 /**
  * Statistics related to a Java VM.
  * This version is hardcoded to use 1.5 MXBean stats from java.lang.management.
  */
 public class VMStats50 implements VMStatsContract {
+  private final static Logger logger = LogService.getLogger(VMStats50.class.getName());
+
   private final static StatisticsType vmType;
 
   private final static ClassLoadingMXBean clBean;
@@ -480,6 +503,7 @@ public class VMStats50 implements VMStatsContract {
   }
 
   private void refreshMemoryPools() {
+    boolean reInitPools = false;
     Iterator<Map.Entry<MemoryPoolMXBean,Statistics>> it = mpMap.entrySet().iterator();
     while (it.hasNext()) {
       Map.Entry<MemoryPoolMXBean,Statistics> me = it.next();
@@ -488,6 +512,7 @@ public class VMStats50 implements VMStatsContract {
       if (!mp.isValid()) {
         s.close();
         it.remove();
+        reInitPools = true;
       } else {
         MemoryUsage mu = null;
         try {
@@ -495,6 +520,14 @@ public class VMStats50 implements VMStatsContract {
         } catch (IllegalArgumentException ex) {
           // to workaround JRockit bug 36348 just ignore this and try the next pool
           continue;
+        } catch (InternalError ie) {
+          // Somebody saw an InternalError once but I have no idea how to reproduce it. Was this a race between
+          // mp.isValid() and mp.getUsage()? Perhaps.
+          s.close();
+          it.remove();
+          reInitPools = true;
+          logger.warn("Accessing MemoryPool '{}' threw an Internal Error: {}", mp.getName(), ie.getMessage());
+          continue;
         }
         s.setLong(mp_l_initMemoryId, mu.getInit());
         s.setLong(mp_l_usedMemoryId, mu.getUsed());
@@ -531,6 +564,9 @@ public class VMStats50 implements VMStatsContract {
         }
       }
     }
+    if (reInitPools) {
+      initMemoryPools();
+    }
   }
 
   private void initGC() {


[07/16] incubator-geode git commit: GEODE-1209: Test changes, removing a sleep call which is not needed...

Posted by kl...@apache.org.
GEODE-1209: Test changes, removing a sleep call which is not needed...

This is in addition to the previous checkin:
Commit: 52a13e8295bdced8bfc72ab7a710f2a4a8df0ca7


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

Branch: refs/heads/feature/GEODE-1565
Commit: 374a1dc40c15c0e3bcb5ad3867090c02257e9013
Parents: 52a13e8
Author: Anil <ag...@pivotal.io>
Authored: Fri Jun 17 11:09:30 2016 -0700
Committer: Anil <ag...@pivotal.io>
Committed: Fri Jun 17 11:09:30 2016 -0700

----------------------------------------------------------------------
 .../AsyncEventQueueEvictionAndExpirationJUnitTest.java          | 5 +----
 1 file changed, 1 insertion(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/374a1dc4/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java
index c37810f..6dfbe05 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueEvictionAndExpirationJUnitTest.java
@@ -380,10 +380,7 @@ public class AsyncEventQueueEvictionAndExpirationJUnitTest {
     region.put("Key-1", "Value-1");
     region.put("Key-2", "Value-2");
     
-    try {
-      Thread.sleep(2000);
-    } catch (Exception ex) {}
-  }
+   }
 
 
   private void waitForAEQEventsNotQueued() {


[03/16] incubator-geode git commit: GEODE-1556: refactor and create unit tests for GFSH hostname

Posted by kl...@apache.org.
GEODE-1556: refactor and create unit tests for GFSH hostname

* extract hostname methods to HostName class
* create HostNameTest
* This closes #165 [klund@apache.org]


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

Branch: refs/heads/feature/GEODE-1565
Commit: 28d2ce069681f3fc246b4f9405869ed51d1007ff
Parents: 8a13222
Author: gmeilen <gr...@gmail.com>
Authored: Thu Jun 16 15:33:53 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Thu Jun 16 15:37:42 2016 -0700

----------------------------------------------------------------------
 .../gemfire/internal/lang/SystemUtils.java      |  11 ++
 .../gemfire/internal/util/HostName.java         |  71 +++++++++++
 .../management/internal/cli/shell/Gfsh.java     |  40 +-----
 .../gemfire/internal/util/HostNameTest.java     | 121 +++++++++++++++++++
 4 files changed, 205 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/28d2ce06/geode-core/src/main/java/com/gemstone/gemfire/internal/lang/SystemUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/lang/SystemUtils.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/lang/SystemUtils.java
index 1c52a10..07ef612 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/lang/SystemUtils.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/lang/SystemUtils.java
@@ -49,6 +49,7 @@ public class SystemUtils {
   public static final String LINUX_OS_NAME = "Linux";
   public static final String MAC_OSX_NAME = "Mac";
   public static final String WINDOWS_OS_NAME = "Windows";
+  public static final String SOLARIS_OS_NAME = "SunOS";
 
   /**
    * Utility method to determine whether the installed Java Runtime Environment (JRE) is minimally at the specified,
@@ -199,6 +200,16 @@ public class SystemUtils {
     return isOS(WINDOWS_OS_NAME);
   }
 
+  /**
+   * Utility method that determines whether the Java application process is executing in a Sun Solaris
+   * operating system environment.
+   *
+   * @return a boolean value indicating whether the Java application process is executing in Solaris.
+   * @see #isOS(String)
+   */
+  public static boolean isSolaris() {
+    return isOS(SOLARIS_OS_NAME);
+  }
 
   /**
    * Returns true if the specified location is in the JVM classpath. This may

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/28d2ce06/geode-core/src/main/java/com/gemstone/gemfire/internal/util/HostName.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/util/HostName.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/util/HostName.java
new file mode 100644
index 0000000..582240f
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/util/HostName.java
@@ -0,0 +1,71 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Scanner;
+
+import com.gemstone.gemfire.internal.lang.SystemUtils;
+
+public class HostName {
+
+  static final String COMPUTER_NAME_PROPERTY = "COMPUTERNAME";
+  static final String HOSTNAME_PROPERTY = "HOSTNAME";
+
+  private static final String HOSTNAME = "hostname";
+  private static final String START_OF_STRING = "\\A";
+  private static final String UNKNOWN = "unknown";
+
+  public String determineHostName() {
+    String hostname = getHostNameFromEnv();
+    if (isEmpty(hostname)) {
+      hostname = execHostName();
+    }
+    assert !isEmpty(hostname);
+    return hostname;
+  }
+
+  String execHostName() {
+    String hostname;
+    try {
+      Process process = new ProcessBuilder(HOSTNAME).start();
+      try (InputStream stream = process.getInputStream();
+           Scanner s = new Scanner(stream).useDelimiter(START_OF_STRING);) {
+        hostname = s.hasNext() ? s.next().trim() : UNKNOWN;
+      }
+    } catch (IOException hostnameBinaryNotFound) {
+      hostname = UNKNOWN;
+    }
+    return hostname;
+  }
+
+  String getHostNameFromEnv() {
+    final String hostname;
+    if (SystemUtils.isWindows()) {
+      hostname = System.getenv(COMPUTER_NAME_PROPERTY);
+    } else {
+      hostname = System.getenv(HOSTNAME_PROPERTY);
+    }
+    return hostname;
+  }
+
+  private boolean isEmpty(String value) {
+    return value == null || value.isEmpty();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/28d2ce06/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/Gfsh.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/Gfsh.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/Gfsh.java
index da13bd9..a6f11bf 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/Gfsh.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/shell/Gfsh.java
@@ -41,6 +41,7 @@ import com.gemstone.gemfire.internal.Banner;
 import com.gemstone.gemfire.internal.GemFireVersion;
 import com.gemstone.gemfire.internal.lang.ClassUtils;
 import com.gemstone.gemfire.internal.process.signal.AbstractSignalNotificationHandler;
+import com.gemstone.gemfire.internal.util.HostName;
 import com.gemstone.gemfire.internal.util.SunAPINotFoundException;
 import com.gemstone.gemfire.management.cli.CommandProcessingException;
 import com.gemstone.gemfire.management.cli.Result;
@@ -280,7 +281,7 @@ public class Gfsh extends JLineShell {
   private void initializeEnvironment() {
     env.put(ENV_SYS_USER,              System.getProperty("user.name"));
     env.put(ENV_SYS_USER_HOME,         System.getProperty("user.home"));
-    env.put(ENV_SYS_HOST_NAME,         determineHostName());
+    env.put(ENV_SYS_HOST_NAME,         new HostName().determineHostName());
     env.put(ENV_SYS_CLASSPATH,         System.getProperty("java.class.path"));
     env.put(ENV_SYS_JAVA_VERSION,      System.getProperty("java.version"));
     env.put(ENV_SYS_OS,                System.getProperty("os.name"));
@@ -295,9 +296,6 @@ public class Gfsh extends JLineShell {
     readonlyAppEnv.add(ENV_APP_LOG_FILE);
     env.put(ENV_APP_PWD,                        System.getProperty("user.dir"));
     readonlyAppEnv.add(ENV_APP_PWD);
-// Enable when "use region" command is required. See #46110
-//    env.put(CliConstants.ENV_APP_CONTEXT_PATH,               CliConstants.DEFAULT_APP_CONTEXT_PATH);
-//    readonlyAppEnv.add(CliConstants.ENV_APP_CONTEXT_PATH);
     env.put(ENV_APP_FETCH_SIZE,                 String.valueOf(DEFAULT_APP_FETCH_SIZE));
     env.put(ENV_APP_LAST_EXIT_STATUS,           String.valueOf(DEFAULT_APP_LAST_EXIT_STATUS));
     readonlyAppEnv.add(ENV_APP_LAST_EXIT_STATUS);
@@ -307,40 +305,6 @@ public class Gfsh extends JLineShell {
     env.put(ENV_APP_RESULT_VIEWER,            String.valueOf(DEFAULT_APP_RESULT_VIEWER));
   }
 
-  private static String execReadToString(String execCommand) throws IOException {
-    Process proc = Runtime.getRuntime().exec(execCommand);
-    try (InputStream stream = proc.getInputStream()) {
-      try (Scanner s = new Scanner(stream).useDelimiter("\\A")) {
-        return s.hasNext() ? s.next().trim() : "";
-      }
-    }
-  }
-
-  private String determineHostName()
-  {
-    // Windows
-    String hostname = System.getenv("COMPUTERNAME");
-    if (hostname == null) {
-      // Unix / Mac / Cygwin sometimes has this set
-      hostname = System.getenv("HOSTNAME");
-    }
-    if (hostname == null) {
-      try {
-        // Unix / Mac / Windows / Cygwin
-        hostname = execReadToString("hostname");
-      }
-      catch (IOException io) {
-        // happens if hostname binary is not found.
-        hostname = "unknown";
-      }
-    }
-    if (hostname == null || hostname.isEmpty()) {
-      // if it still isn't set, default it.
-      hostname = "unknown";
-    }
-    return hostname;
-  }
-
   public static Gfsh getInstance(boolean launchShell, String[] args, GfshConfig gfshConfig)
     throws ClassNotFoundException, IOException
   {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/28d2ce06/geode-core/src/test/java/com/gemstone/gemfire/internal/util/HostNameTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/util/HostNameTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/HostNameTest.java
new file mode 100644
index 0000000..98a87fc
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/util/HostNameTest.java
@@ -0,0 +1,121 @@
+/*
+ * 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.util;
+
+import static com.gemstone.gemfire.internal.lang.SystemUtils.*;
+import static com.gemstone.gemfire.internal.util.HostName.*;
+import static org.assertj.core.api.Assertions.*;
+
+import java.io.IOException;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.EnvironmentVariables;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+
+@Category(UnitTest.class)
+@RunWith(JUnitParamsRunner.class)
+public class HostNameTest {
+
+  private static final String EXPECTED_HOSTNAME = "expected-hostname";
+  private static final String UNKNOWN = "unknown";
+
+  @Rule
+  public final EnvironmentVariables env = new EnvironmentVariables();
+
+  @Rule
+  public final RestoreSystemProperties sysProps = new RestoreSystemProperties();
+
+  @Test
+  public void execHostNameShouldNeverReturnNull() throws IOException {
+    String result = new HostName().execHostName();
+    assertThat(result).isNotNull();
+  }
+
+  @Test
+  @Parameters({ MAC_OSX_NAME, LINUX_OS_NAME, SOLARIS_OS_NAME, WINDOWS_OS_NAME })
+  public void shouldExecHostNameIfEnvValueNotAvailableOnOS(String osName) throws IOException {
+    setHostNamePropertiesNull(osName);
+    String result = new HostName().determineHostName();
+    assertThat(result).isNotNull();
+  }
+
+  @Test
+  @Parameters({ MAC_OSX_NAME, LINUX_OS_NAME, SOLARIS_OS_NAME, WINDOWS_OS_NAME })
+  public void shouldUseComputerNameIfAvailableOnOS(String osName) throws IOException {
+    setHostNameProperties(osName);
+    String result = new HostName().determineHostName();
+    assertThat(result).isEqualTo(EXPECTED_HOSTNAME);
+  }
+
+  @Test
+  @Parameters({ MAC_OSX_NAME, LINUX_OS_NAME, SOLARIS_OS_NAME, WINDOWS_OS_NAME })
+  public void shouldBeNullIfEnvValueNotAvailableOnOS(String osName) throws IOException {
+    setHostNamePropertiesNull(osName);
+    String result = new HostName().getHostNameFromEnv();
+    assertThat(result).isEqualTo(null);
+  }
+
+  private void setHostNameProperties(String osName) {
+    System.setProperty("os.name", osName);
+    if (isWindows()) {
+      this.env.set(COMPUTER_NAME_PROPERTY, EXPECTED_HOSTNAME);
+      this.env.set(HOSTNAME_PROPERTY, null);
+    } else {
+      this.env.set(COMPUTER_NAME_PROPERTY, null);
+      this.env.set(HOSTNAME_PROPERTY, EXPECTED_HOSTNAME);
+    }
+
+    assertThat(System.getProperty("os.name")).isEqualTo(osName);
+    if (isWindows()) {
+      assertThat(System.getenv(COMPUTER_NAME_PROPERTY)).isEqualTo(EXPECTED_HOSTNAME);
+      assertThat(System.getenv(HOSTNAME_PROPERTY)).isNull();
+    } else {
+      assertThat(System.getenv(COMPUTER_NAME_PROPERTY)).isNull();
+      assertThat(System.getenv(HOSTNAME_PROPERTY)).isEqualTo(EXPECTED_HOSTNAME);
+    }
+  }
+
+  private void setHostNamePropertiesNull(String osName) {
+    System.setProperty("os.name", osName);
+    if (isWindows()) {
+      this.env.set(COMPUTER_NAME_PROPERTY, null);
+      this.env.set(HOSTNAME_PROPERTY, null);
+    } else {
+      this.env.set(COMPUTER_NAME_PROPERTY, null);
+      this.env.set(HOSTNAME_PROPERTY, null);
+    }
+
+    assertThat(System.getProperty("os.name")).isEqualTo(osName);
+    if (isWindows()) {
+      assertThat(System.getenv(COMPUTER_NAME_PROPERTY)).isNull();
+      assertThat(System.getenv(HOSTNAME_PROPERTY)).isNull();
+    } else {
+      assertThat(System.getenv(COMPUTER_NAME_PROPERTY)).isNull();
+      assertThat(System.getenv(HOSTNAME_PROPERTY)).isNull();
+    }
+  }
+
+
+}


[09/16] incubator-geode git commit: GEODE-835: remove developer name from comments

Posted by kl...@apache.org.
GEODE-835: remove developer name from comments


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

Branch: refs/heads/feature/GEODE-1565
Commit: 947628a202f8caa197fd2911257bef0dd754a201
Parents: 43a9918
Author: Kirk Lund <kl...@apache.org>
Authored: Fri Jun 17 11:38:35 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Jun 17 13:19:47 2016 -0700

----------------------------------------------------------------------
 .../internal/cli/parser/jopt/JoptOptionParser.java        | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/947628a2/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/parser/jopt/JoptOptionParser.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/parser/jopt/JoptOptionParser.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/parser/jopt/JoptOptionParser.java
index bbda3e6..04590ed 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/parser/jopt/JoptOptionParser.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/parser/jopt/JoptOptionParser.java
@@ -115,7 +115,7 @@ public class JoptOptionParser implements GfshOptionParser {
     optionSet.setUserInput(userInput!=null?userInput.trim():"");
     if (userInput != null) {
       TrimmedInput input = PreprocessorUtils.trim(userInput);
-      String[] preProcessedInput = preProcess(new OptionJFormatter().formatCommand(input.getString())); // TODO: use OptionJFormatter
+      String[] preProcessedInput = preProcess(new OptionJFormatter().formatCommand(input.getString()));
       joptsimple.OptionSet joptOptionSet = null;
       CliCommandOptionException ce = null;
       // int factor = 0;
@@ -123,21 +123,21 @@ public class JoptOptionParser implements GfshOptionParser {
         joptOptionSet = parser.parse(preProcessedInput);
       } catch (OptionException e) {
         ce = processException(e);
-        // TODO:KIRK: joptOptionSet = e.getDetected(); // TODO: removed when geode-joptsimple was removed
+        // TODO: joptOptionSet = e.getDetected(); // removed when geode-joptsimple was removed
       }
       if (joptOptionSet != null) {
 
         // Make sure there are no miscellaneous, unknown strings that cannot be identified as
         // either options or arguments.
         if (joptOptionSet.nonOptionArguments().size() > arguments.size()) {
-          String unknownString = (String)joptOptionSet.nonOptionArguments().get(arguments.size()); // TODO: added cast when geode-joptsimple was removed
+          String unknownString = (String)joptOptionSet.nonOptionArguments().get(arguments.size()); // added cast when geode-joptsimple was removed
           // If the first option is un-parseable then it will be returned as "<option>=<value>" since it's
           // been interpreted as an argument. However, all subsequent options will be returned as "<option>".
           // This hack splits off the string before the "=" sign if it's the first case.
           if (unknownString.matches("^-*\\w+=.*$")) {
             unknownString = unknownString.substring(0, unknownString.indexOf('='));
           }
-          // TODO:KIRK: ce = processException(OptionException.createUnrecognizedOptionException(unknownString, joptOptionSet)); // TODO: removed when geode-joptsimple was removed
+          // TODO: ce = processException(OptionException.createUnrecognizedOptionException(unknownString, joptOptionSet)); // removed when geode-joptsimple was removed
         }
         
         // First process the arguments
@@ -199,7 +199,7 @@ public class JoptOptionParser implements GfshOptionParser {
                   if (arguments.size() > 1 && !(option.getConverter() instanceof MultipleValueConverter) && option.getValueSeparator() == null) {
                     List<String> optionList = new ArrayList<String>(1);
                     optionList.add(string);
-                    // TODO:KIRK: ce = processException(new MultipleArgumentsForOptionException(optionList, joptOptionSet)); // TODO: removed when geode-joptsimple was removed
+                    // TODO: ce = processException(new MultipleArgumentsForOptionException(optionList, joptOptionSet)); // removed when geode-joptsimple was removed
                   } else if ((arguments.size() == 1 && !(option.getConverter() instanceof MultipleValueConverter)) || option.getValueSeparator() == null) {
                     optionSet.put(option, arguments.get(0).toString().trim());
                   } else {


[08/16] incubator-geode git commit: GEODE-1559: add javadocs to OptionJFormatter

Posted by kl...@apache.org.
GEODE-1559: add javadocs to OptionJFormatter


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

Branch: refs/heads/feature/GEODE-1565
Commit: 43a9918f571822d807feb68f14ca59dc87c89001
Parents: bcffff9
Author: Kirk Lund <kl...@apache.org>
Authored: Fri Jun 17 11:34:25 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Jun 17 13:19:47 2016 -0700

----------------------------------------------------------------------
 .../management/internal/cli/util/OptionJFormatter.java   | 11 ++++++++++-
 1 file changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/43a9918f/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/OptionJFormatter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/OptionJFormatter.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/OptionJFormatter.java
index 4534f19..0e3265a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/OptionJFormatter.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/OptionJFormatter.java
@@ -16,6 +16,9 @@
  */
 package com.gemstone.gemfire.management.internal.cli.util;
 
+/**
+ * Inserts quotes around the values of any --J options.
+ */
 public class OptionJFormatter {
 
   private static final String J_OPTION = "--J=";
@@ -27,6 +30,9 @@ public class OptionJFormatter {
   private String command;
   private StringBuilder formatted;
 
+  /**
+   * Returns command with quotes inserted around the values of any --J options.
+   */
   public String formatCommand(String command){
     if (!containsJopt(command)) {
       return command;
@@ -96,7 +102,10 @@ public class OptionJFormatter {
     return formatted.toString();
   }
 
-  public boolean containsJopt(String cmd){
+  /**
+   * Returns true if command contains any --J options.
+   */
+  boolean containsJopt(String cmd){
     if (cmd.contains("--J")){
       return true;
     }